Runtime Partition Pruning

Started by Beena Emersonover 8 years ago186 messages
#1Beena Emerson
memissemerson@gmail.com
1 attachment(s)

I have been working on implementing the runtime partition pruning
which would increase the performance of queries involving partitioned
table to a great extent.

PFA the POC which can be applied over Amit's patch for faster
partition pruning [1]/messages/by-id/e02923ea-a117-a6ad-6a3e-ea5e1ba41ece@lab.ntt.co.jp and Dilip's refactor patch [2]/messages/by-id/CAFiTN-tGnQzF_4QtbOHT-3hE=OvNaMfbbeRxa4UY0CQyF0G8gQ@mail.gmail.com on commit
2c74e6c1dcc5002fa8b822e5757f6c95d899fb7a.

[1]: /messages/by-id/e02923ea-a117-a6ad-6a3e-ea5e1ba41ece@lab.ntt.co.jp

[2]: /messages/by-id/CAFiTN-tGnQzF_4QtbOHT-3hE=OvNaMfbbeRxa4UY0CQyF0G8gQ@mail.gmail.com

There were a couple of things that need improvement/opinion:
In get_rel_partition_info, we store minop and maxop for each partition
key. For the equality case, which is most common, both would store the
same value. We could make it better by storing equal (bound, bound,
....) instead repeating the same values.

get_partitions_for_keys currently returns the list of partitions valid
for the given keys but for a table with many partitions this list
would be very long so maybe for range qual ( key > a & key < b ) we
could only store the min and max partition number and increment
as_whichplan by 1 till we reach max partition number. For
non-continuous partitions, we would still need the list.

Currently, the partitions numbers are recalculated whenever the
ChgParam is set, This can be optimised by skipping this step when only
a non-partition key column has changed; reusing the existing
partitions selected.

Others:
- better handling of multiple key
- allow use of expression in the quals.
- To use min_incl, max_incl properly in get_partitions_for_keys.
- pruning during function calls.

Currently with patch, during NestLoop:
Nested Loop
-> SeqScan tbl1
-> Append
-> Index Scan p01
-> Index Scan p02
-> Index Scan p03

For each tuple from tbl1, only the relevant partition (p01or p02 or
p03) will be scanned.

--- Prepared Statement Behaviour with patch---

Table Descritpion:
Table "public.tprt"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
col1 | integer | | | | plain | |
col2 | integer | | | | plain | |
Partition key: RANGE (col1)
Partitions: tprt_1 FOR VALUES FROM (1) TO (50001),
tprt_2 FOR VALUES FROM (50001) TO (100001),
tprt_3 FOR VALUES FROM (100001) TO (200001)

EXPLAIN EXECUTE prstmt_select(15);

QUERY PLAN
------------------------------------------------------------------
Append (cost=0.00..1736.55 rows=1 width=8)
-> Seq Scan on tprt_1 (cost=0.00..849.15 rows=16724 width=8)
Filter: (col1 < $1)
(3 rows)

EXPLAIN EXECUTE prstmt_select(60000);
QUERY PLAN
------------------------------------------------------------------
Append (cost=0.00..1736.55 rows=2 width=8)
-> Seq Scan on tprt_1 (cost=0.00..849.15 rows=16724 width=8)
Filter: (col1 < $1)
-> Seq Scan on tprt_2 (cost=0.00..849.15 rows=16724 width=8)
Filter: (col1 < $1)
(5 rows)

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-POC-Implement-runtime-partiton-pruning.patchapplication/octet-stream; name=0001-POC-Implement-runtime-partiton-pruning.patchDownload
From ae31ae294cef9660fcc547abfc7543b5b3c85a1f Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Tue, 26 Sep 2017 09:26:28 +0530
Subject: [PATCH] Implement runtime partiton pruning

Patch by: Beena Emerson, Dilip Kumar
---
 src/backend/executor/nodeAppend.c       | 179 +++++++++++++++++++++++++++++++-
 src/backend/nodes/copyfuncs.c           |  22 ++++
 src/backend/nodes/outfuncs.c            |  14 +++
 src/backend/nodes/readfuncs.c           |  18 ++++
 src/backend/optimizer/path/allpaths.c   | 171 ++++++++++++++++++++++++++++--
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  17 +++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  17 +++
 src/backend/optimizer/util/pathnode.c   |  38 ++++++-
 src/backend/optimizer/util/relnode.c    |  24 ++++-
 src/backend/utils/cache/plancache.c     |   2 +-
 src/include/nodes/execnodes.h           |  13 +++
 src/include/nodes/nodes.h               |   1 +
 src/include/nodes/plannodes.h           |  13 +++
 src/include/nodes/relation.h            |  13 +++
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   2 +-
 src/include/optimizer/paths.h           |   3 +
 20 files changed, 535 insertions(+), 21 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index bed9bb8..19587d4 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "nodes/relation.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "optimizer/clauses.h"
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool exec_append_initialize_next(AppendState *appendstate);
@@ -165,6 +167,86 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->et_rtp_list)
+	{
+		List	   *temp_appendplans = NIL;
+		Append	   *temp_node;
+
+		Datum	   *minkeys,
+				   *maxkeys;
+		Relation	rel;
+		NullTestType keynullness = IS_NOT_NULL;
+		int			min_datum_index,
+					max_datum_index;
+		bool		null_partition_chosen = false;
+		bool		default_partition_chosen = false;
+		int			i = 0;
+		int			n_minkeys = 0,
+					n_maxkeys = 0;
+
+		minkeys = (Datum *) palloc(list_length(node->et_rtp_list) * sizeof(Datum));
+		maxkeys = (Datum *) palloc(list_length(node->et_rtp_list) * sizeof(Datum));
+
+		foreach(lc, node->et_rtp_list)
+		{
+			RuntimePruningInfo *rtp = (RuntimePruningInfo *) lfirst(lc);
+			RuntimePruningExec *rtpe = palloc0(sizeof(RuntimePruningExec));
+
+			if (rtp->maxop)
+			{
+				Node	   *n = eval_const_expressions2(estate->es_param_list_info, (Node *) rtp->maxop);
+
+				maxkeys[i] = ((Const *) n)->constvalue;
+				n_maxkeys++;
+			}
+			if (rtp->minop)
+			{
+				Node	   *n = eval_const_expressions2(estate->es_param_list_info, (Node *) rtp->minop);
+
+				minkeys[i] = ((Const *) n)->constvalue;
+				n_minkeys++;
+			}
+			rtpe->minincl = rtp->minincl;
+			rtpe->maxincl = rtp->maxincl;
+
+			i++;
+		}
+
+		rel = relation_open(node->parentoid, NoLock);
+		appendstate->index = get_partitions_for_keys(rel,
+													 &keynullness,
+													 minkeys, n_minkeys, false,
+													 maxkeys, n_maxkeys, false,
+													 &min_datum_index, &max_datum_index,
+													 &null_partition_chosen,
+													 &default_partition_chosen);
+		relation_close(rel, NoLock);
+
+		i = 0;
+		foreach(lc, appendstate->index)
+		{
+			int			index = lfirst_int(lc);
+
+			Plan	   *initNode = (Plan *) list_nth(node->appendplans, index);
+
+			temp_appendplans = lappend(temp_appendplans, initNode);
+			appendplanstates[i++] = ExecInitNode(initNode, estate, eflags);
+		}
+
+		/* create new AppendState for our append node */
+		temp_node = copyObject(node);
+		temp_node->appendplans = temp_appendplans;
+		((Plan *) temp_node)->plan_rows = i;
+		appendstate->as_nplans = i;
+		appendstate->ps.plan = (Plan *) temp_node;
+		appendstate->ps.state = estate;
+		appendstate->ps.ExecProcNode = ExecAppend;
+		appendstate->appendplans = (PlanState **) palloc0(i * sizeof(PlanState *));
+		appendstate->appendplans = appendplanstates;
+
+		return appendstate;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -190,6 +272,27 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->as_whichplan = 0;
 	exec_append_initialize_next(appendstate);
 
+	if (node->rtp_list)
+	{
+		ListCell   *lc;
+
+		foreach(lc, node->rtp_list)
+		{
+			RuntimePruningInfo *rtp = (RuntimePruningInfo *) lfirst(lc);
+			RuntimePruningExec *rtpe = palloc0(sizeof(RuntimePruningExec));
+
+			rtpe->maxop = ExecInitExpr(rtp->maxop, appendplanstates);
+			rtpe->minop = ExecInitExpr(rtp->minop, appendplanstates);
+			rtpe->minincl = rtp->minincl;
+			rtpe->maxincl = rtp->maxincl;
+
+			appendstate->rtp_list = lappend(appendstate->rtp_list, rtpe);
+		}
+		appendstate->parentoid = node->parentoid;
+		ExecAssignExprContext(estate, &appendstate->ps);
+	}
+
+
 	return appendstate;
 }
 
@@ -237,9 +340,21 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/* For runtime partition pruning, goto the next valid partition index*/
+			if (node->index)
+			{
+				if (++node->as_whichpartition < list_length(node->index))
+					node->as_whichplan = list_nth_int(node->index, node->as_whichpartition);
+
+				return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
+
 		if (!exec_append_initialize_next(node))
 			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
 
@@ -279,7 +394,60 @@ void
 ExecReScanAppend(AppendState *node)
 {
 	int			i;
+	ListCell   *lc;
+	int			n_minkeys = 0;
+	int			n_maxkeys = 0;
 
+	if (node->ps.chgParam != NULL && node->rtp_list)
+	{
+		Datum	   *minkeys,
+				   *maxkeys;
+		bool		maxisnull,
+					minisnull;
+		Relation	rel;
+		NullTestType keynullness = IS_NOT_NULL;
+		int			min_datum_index,
+					max_datum_index;
+		bool		null_partition_chosen = false;
+		bool		default_partition_chosen = false;
+		int			i;
+
+		i = 0;
+		minkeys = (Datum *) palloc(list_length(node->rtp_list) * sizeof(Datum));
+		maxkeys = (Datum *) palloc(list_length(node->rtp_list) * sizeof(Datum));
+
+		foreach(lc, node->rtp_list)
+		{
+			RuntimePruningExec *rtpe = (RuntimePruningExec *) lfirst(lc);
+
+			if (rtpe->minop)
+			{
+				minkeys[i] = ExecEvalExpr(rtpe->minop, node->ps.ps_ExprContext, &minisnull);
+				n_minkeys++;
+			}
+			if (rtpe->maxop)
+			{
+				maxkeys[i] = ExecEvalExpr(rtpe->maxop, node->ps.ps_ExprContext, &maxisnull);
+				n_maxkeys++;
+			}
+			i++;
+
+			if (node->parentoid)
+			{
+				rel = relation_open(node->parentoid, NoLock);
+
+				/* FIXME:  min_incl and max_incl is currently taken as false */
+				node->index = get_partitions_for_keys(rel,
+													  &keynullness,
+													  minkeys, n_minkeys, false,
+													  maxkeys, n_maxkeys, false,
+													  &min_datum_index, &max_datum_index,
+													  &null_partition_chosen,
+													  &default_partition_chosen);
+				relation_close(rel, NoLock);
+			}
+		}
+	}
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +466,13 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->index)
+	{
+		node->as_whichplan = linitial_int(node->index);
+		node->as_whichpartition = 0;
+	}
+	else
+		node->as_whichplan = 0;
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1bed14..a1562f3 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,9 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(rtp_list);
+	COPY_NODE_FIELD(et_rtp_list);
+	COPY_SCALAR_FIELD(parentoid);
 
 	return newnode;
 }
@@ -1145,6 +1148,22 @@ _copyNestLoopParam(const NestLoopParam *from)
 }
 
 /*
+ * _copyRuntimePruningInfo,
+ */
+static RuntimePruningInfo *
+_copyRuntimePruningInfo(const RuntimePruningInfo *from)
+{
+	RuntimePruningInfo *newnode = makeNode(RuntimePruningInfo);
+
+	COPY_NODE_FIELD(minop);
+	COPY_NODE_FIELD(maxop);
+	COPY_SCALAR_FIELD(minincl);
+	COPY_SCALAR_FIELD(maxincl);
+
+	return newnode;
+}
+
+/*
  * _copyPlanRowMark
  */
 static PlanRowMark *
@@ -4816,6 +4835,9 @@ copyObjectImpl(const void *from)
 		case T_NestLoopParam:
 			retval = _copyNestLoopParam(from);
 			break;
+		case T_RuntimePruningInfo:
+			retval = _copyRuntimePruningInfo(from);
+			break;
 		case T_PlanRowMark:
 			retval = _copyPlanRowMark(from);
 			break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b83d919..5764c7a 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -978,6 +978,17 @@ _outNestLoopParam(StringInfo str, const NestLoopParam *node)
 }
 
 static void
+_outRuntimePruningInfo(StringInfo str, const RuntimePruningInfo *node)
+{
+	WRITE_NODE_TYPE("RUNTIMEPRUNINGINFO");
+
+	WRITE_NODE_FIELD(minop);
+	WRITE_NODE_FIELD(maxop);
+	WRITE_BOOL_FIELD(maxincl);
+	WRITE_BOOL_FIELD(minincl);
+}
+
+static void
 _outPlanRowMark(StringInfo str, const PlanRowMark *node)
 {
 	WRITE_NODE_TYPE("PLANROWMARK");
@@ -3746,6 +3757,9 @@ outNode(StringInfo str, const void *obj)
 			case T_NestLoopParam:
 				_outNestLoopParam(str, obj);
 				break;
+			case T_RuntimePruningInfo:
+				_outRuntimePruningInfo(str, obj);
+				break;
 			case T_PlanRowMark:
 				_outPlanRowMark(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fbf8330..b090837 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2277,6 +2277,22 @@ _readNestLoopParam(void)
 }
 
 /*
+ * _readRuntimePruningInfo
+ */
+static RuntimePruningInfo *
+_readRuntimePruningInfo(void)
+{
+	READ_LOCALS(RuntimePruningInfo);
+
+	READ_NODE_FIELD(minop);
+	READ_NODE_FIELD(maxop);
+	READ_BOOL_FIELD(minincl);
+	READ_BOOL_FIELD(maxincl);
+
+	READ_DONE();
+}
+
+/*
  * _readPlanRowMark
  */
 static PlanRowMark *
@@ -2646,6 +2662,8 @@ parseNodeString(void)
 		return_value = _readLimit();
 	else if (MATCH("NESTLOOPPARAM", 13))
 		return_value = _readNestLoopParam();
+	else if (MATCH("RUNTIMEPRUNINGINFO", 18))
+		return_value = _readRuntimePruningInfo();
 	else if (MATCH("PLANROWMARK", 11))
 		return_value = _readPlanRowMark();
 	else if (MATCH("PLANINVALITEM", 13))
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 2bb3641..c34826d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -848,7 +848,7 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 
 static void
 get_matching_clause(RelOptInfo *rel, List *clauses, List **matchedclauses,
-					NullTestType *keynullness)
+					NullTestType *keynullness, bool is_extern)
 {
 	ListCell   *lc;
 	int		keyPos;
@@ -883,6 +883,8 @@ get_matching_clause(RelOptInfo *rel, List *clauses, List **matchedclauses,
 
 				if (equal(leftop, partkey))
 				{
+					if (is_extern && !IsA(rightop, Param))
+						continue;
 					matchedclauses[keyPos] = lappend(matchedclauses[keyPos],
 													 clause);
 					/* A strict operator implies NOT NULL argument. */
@@ -892,6 +894,9 @@ get_matching_clause(RelOptInfo *rel, List *clauses, List **matchedclauses,
 				{
 					Oid		commutator = get_commutator(expr_op);
 
+					if (is_extern && !IsA(leftop, Param))
+						continue;
+
 					if (OidIsValid(commutator))
 					{
 						OpExpr   *commutated_expr;
@@ -929,6 +934,143 @@ get_matching_clause(RelOptInfo *rel, List *clauses, List **matchedclauses,
 }
 
 /*
+ * FIXME, this function can reuse most of the code from get_rel_partitions. we
+ * can make get_rel_partitions function generic so that it can accept the
+ * clause instead of processing all the baserelrestriction info, this way this
+ * function will be more generic and can be used for appendrel as well as for
+ * baserel.
+ */
+List *
+get_rel_partition_info(RelOptInfo *rel, RangeTblEntry *rte, List *clauses,
+					   RuntimePruningInfo * *prtp, bool is_extern)
+{
+	Relation	parent = heap_open(rte->relid, NoLock);
+	ListCell   *lc1;
+	List	   *matchedclauses[PARTITION_MAX_KEYS];
+	NullTestType keynullness[PARTITION_MAX_KEYS];
+	bool		need_next_min,
+				need_next_max,
+				minkey_set[PARTITION_MAX_KEYS],
+				maxkey_set[PARTITION_MAX_KEYS],
+				min_incl,
+				max_incl;
+	int			n_minkeys = 0,
+				n_maxkeys = 0,
+				i;
+
+	List	   *rtp_list = NIL;
+
+	get_matching_clause(rel, clauses, matchedclauses, keynullness, is_extern);
+
+	/*
+	 * Determine the min keys and the max keys using btree semantics-based
+	 * interpretation of the clauses' operators.
+	 */
+
+	/*
+	 * XXX - There should be a step similar to _bt_preprocess_keys() here, to
+	 * eliminate any redundant scan keys for a given partition column.  For
+	 * example, among a <= 4 and a <= 5, we can only keep a <= 4 for being
+	 * more restrictive and discard a <= 5.  While doing that, we can also
+	 * check to see if there exists a contradictory combination of scan keys
+	 * that makes the query trivially false for all records in the table.
+	 */
+	memset(minkey_set, false, sizeof(minkey_set));
+	memset(maxkey_set, false, sizeof(maxkey_set));
+	need_next_min = true;
+	need_next_max = true;
+	for (i = 0; i < rel->part_scheme->partnatts; i++)
+	{
+		/*
+		 * If no scan key existed for the previous column, we are done.
+		 */
+		if (i > n_minkeys)
+			need_next_min = false;
+
+		if (i > n_maxkeys)
+			need_next_max = false;
+
+		foreach(lc1, matchedclauses[i])
+		{
+			Expr	   *clause = lfirst(lc1);
+			Expr	   *leftop = (Expr *) get_leftop(clause);
+			Expr	   *rightop = (Expr *) get_rightop(clause);
+			Oid			opno = ((OpExpr *) clause)->opno,
+						opfamily = rel->part_scheme->partopfamily[i];
+			StrategyNumber strategy;
+			RuntimePruningInfo *rtp_temp = makeNode(RuntimePruningInfo);
+
+			strategy = get_op_opfamily_strategy(opno, opfamily);
+			switch (strategy)
+			{
+				case BTLessStrategyNumber:
+				case BTLessEqualStrategyNumber:
+					if (need_next_max)
+					{
+						rtp_temp->maxop = rightop;
+						if (!maxkey_set[i])
+							n_maxkeys++;
+						maxkey_set[i] = true;
+						max_incl = (strategy == BTLessEqualStrategyNumber);
+					}
+					if (strategy == BTLessStrategyNumber)
+						need_next_max = false;
+					break;
+
+				case BTGreaterStrategyNumber:
+				case BTGreaterEqualStrategyNumber:
+					if (need_next_min)
+					{
+						rtp_temp->minop = rightop;
+						if (!minkey_set[i])
+							n_minkeys++;
+						minkey_set[i] = true;
+						min_incl = (strategy == BTGreaterEqualStrategyNumber);
+					}
+					if (strategy == BTGreaterStrategyNumber)
+						need_next_min = false;
+					break;
+
+				case BTEqualStrategyNumber:
+					if (need_next_min)
+					{
+						rtp_temp->minop = rightop;
+						if (!minkey_set[i])
+							n_minkeys++;
+					}
+					minkey_set[i] = true;
+					min_incl = true;
+
+					if (need_next_max)
+					{
+						rtp_temp->maxop = rightop;
+						if (!maxkey_set[i])
+							n_maxkeys++;
+					}
+					maxkey_set[i] = true;
+					max_incl = true;
+					break;
+
+					/*
+					 * This might mean '<>', but we don't have anything for
+					 * that case yet.  Perhaps, handle that as key < const OR
+					 * key > const, once we have props needed for handling OR
+					 * clauses.
+					 */
+				default:
+					min_incl = max_incl = false;
+					break;
+			}
+
+			rtp_list = lappend(rtp_list, rtp_temp);
+		}
+	}
+
+	heap_close(parent, NoLock);
+	return rtp_list;
+}
+
+/*
  * get_rel_partitions
  *		Return the list of partitions of rel that pass the query clauses
  *
@@ -938,6 +1080,7 @@ static List *
 get_rel_partitions(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 {
 	Relation		parent = heap_open(rte->relid, NoLock);
+	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
 	List   *indexes;
 	List   *result = NIL;
 	ListCell   *lc1;
@@ -956,7 +1099,7 @@ get_rel_partitions(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 			i;
 
 	get_matching_clause(rel, rel->baserestrictinfo, matchedclauses,
-						keynullness);
+						keynullness, false);
 
 	/*
 	 * Determine the min keys and the max keys using btree semantics-based
@@ -990,11 +1133,17 @@ get_rel_partitions(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 
 		foreach(lc1, matchedclauses[i])
 		{
-			Expr   *clause = lfirst(lc1);
-			Const  *rightop = (Const *) get_rightop(clause);
-			Oid		opno = ((OpExpr *) clause)->opno,
-					opfamily = rel->part_scheme->partopfamily[i];
-			StrategyNumber	strategy;
+			Expr	   *clause = lfirst(lc1);
+			Expr	   *rightop_expr = (Expr *) get_rightop(clause);
+			Oid			opno = ((OpExpr *) clause)->opno,
+						opfamily = rel->part_scheme->partopfamily[i];
+			StrategyNumber strategy;
+			Const	   *rightop;
+
+			if (IsA(rightop_expr, Param))
+				continue;
+
+			rightop = (Const *) rightop_expr;
 
 			strategy = get_op_opfamily_strategy(opno, opfamily);
 			switch (strategy)
@@ -1720,7 +1869,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1747,7 +1896,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1801,7 +1950,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2038,7 +2187,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6ee2350..4b58eb4 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 2821662..991cb64 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -991,6 +991,7 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1053,6 +1054,22 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	if (best_path->rtp_list)
+	{
+		ListCell   *lc;
+
+		foreach(lc, best_path->rtp_list)
+		{
+			RuntimePruningInfo *rtp = (RuntimePruningInfo *) lfirst(lc);
+
+			rtp->minop = (Expr *) replace_nestloop_params(root, (Node *) rtp->minop);
+			rtp->maxop = (Expr *) replace_nestloop_params(root, (Node *) rtp->maxop);
+		}
+		plan->rtp_list = best_path->rtp_list;
+	}
+
+	plan->et_rtp_list = best_path->et_rtp_list;
+	plan->parentoid = best_path->parentoid;
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 7f146d6..0e2d05f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3643,7 +3643,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 3e0c3de..88b0033 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 93add27..15620a0 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -189,6 +189,22 @@ make_opclause(Oid opno, Oid opresulttype, bool opretset,
 	return (Expr *) expr;
 }
 
+Node *
+eval_const_expressions2(ParamListInfo prmList, Node *node)
+{
+	eval_const_expressions_context context;
+
+	if (prmList)
+		context.boundParams = prmList;	/* bound Params */
+	else
+		context.boundParams = NULL;
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
 /*
  * get_leftop
  *
@@ -2507,6 +2523,7 @@ eval_const_expressions_mutator(Node *node,
 								pval = prm->value;
 							else
 								pval = datumCopy(prm->value, typByVal, typLen);
+
 							return (Node *) makeConst(param->paramtype,
 													  param->paramtypmod,
 													  param->paramcollid,
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 26567cb..632105e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -16,6 +16,7 @@
 
 #include <math.h>
 
+#include "catalog/pg_class.h"
 #include "miscadmin.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
@@ -27,6 +28,7 @@
 #include "optimizer/var.h"
 #include "parser/parsetree.h"
 #include "utils/lsyscache.h"
+#include "utils/rel.h"
 #include "utils/selfuncs.h"
 
 
@@ -1200,7 +1202,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1245,6 +1247,40 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	if (required_outer || rel->baserestrictinfo)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+		List	   *query_quals = rel->baserestrictinfo;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = RelationIdGetRelation(poid);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				RuntimePruningInfo *rtp;
+				ParamPathInfo *ppi = pathnode->path.param_info;
+
+				if (ppi)
+				{
+					List	   *ppi_clauses = ppi->ppi_clauses;
+
+					pathnode->rtp_list = get_rel_partition_info(rel, rte, ppi_clauses, &rtp, false);
+				}
+				/* EXTERN PARAM */
+				if (query_quals)
+				{
+					pathnode->et_rtp_list = get_rel_partition_info(rel, rte, query_quals, &rtp, true);
+				}
+				pathnode->parentoid = poid;
+
+			}
+		}
+
+
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index f0973b8..d9cc3f8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1346,6 +1346,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1357,11 +1360,30 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+
+	/*
+	 * FIXME:  Generally for appendrel we don't fetch the clause from the the
+	 * join clause (only we do so for baserel), but for identifying whether
+	 * the appendrel is applicable for runtime pruning or not we need clause
+	 * also, I think there should be better way to do that.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index ad8a82f..458767d 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1041,7 +1041,7 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	if (plansource->num_custom_plans < 5)
 		return true;
 
-	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans;
+	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans + 100000;
 
 	/*
 	 * Prefer generic plan if it's less expensive than the average custom
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index c6d3021..e7de368 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -990,6 +990,14 @@ typedef struct ModifyTableState
 	/* Per plan/partition tuple conversion */
 } ModifyTableState;
 
+typedef struct RuntimePruningExec
+{
+	ExprState  *minop;
+	ExprState  *maxop;
+	bool		minincl;
+	bool		maxincl;
+}			RuntimePruningExec;
+
 /* ----------------
  *	 AppendState information
  *
@@ -1003,6 +1011,11 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Oid			parentoid;
+
+	int			as_whichpartition;
+	List	   *index;
+	List	   *rtp_list;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 63196a1..2c16f8b 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -87,6 +87,7 @@ typedef enum NodeTag
 	T_NestLoopParam,
 	T_PlanRowMark,
 	T_PlanInvalItem,
+	T_RuntimePruningInfo,
 
 	/*
 	 * TAGS FOR PLAN STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index a382331..c10d554 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -237,6 +237,15 @@ typedef struct ModifyTable
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } ModifyTable;
 
+typedef struct RuntimePruning
+{
+	Expr	   *minop;
+	Expr	   *maxop;
+	bool		minincl;
+	bool		maxincl;
+	Oid			parentoid;
+}			RuntimePruning;
+
 /* ----------------
  *	 Append node -
  *		Generate the concatenation of the results of sub-plans.
@@ -248,6 +257,10 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+	Oid			parentoid;
+	RuntimePruning *rtp;
+	List	   *rtp_list;
+	List	   *et_rtp_list;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 0f4996b..f1a2234 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1246,6 +1246,15 @@ typedef struct CustomPath
 	const struct CustomPathMethods *methods;
 } CustomPath;
 
+typedef struct RuntimePruningInfo
+{
+	NodeTag		type;
+	Expr	   *minop;
+	Expr	   *maxop;
+	bool		minincl;
+	bool		maxincl;
+}			RuntimePruningInfo;
+
 /*
  * AppendPath represents an Append plan, ie, successive execution of
  * several member plans.
@@ -1261,6 +1270,10 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+	bool		runtime_prunable;
+	Oid			parentoid;
+	List	   *rtp_list;
+	List	   *et_rtp_list;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index e367221..c19ee31 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions2(ParamListInfo prm_list, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index e372f88..4bcc3c7 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,7 +63,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths,
 				   Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 4e06b2e..b89cb95 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -59,6 +59,9 @@ extern int compute_parallel_worker(RelOptInfo *rel, double heap_pages,
 extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
 							Path *bitmapqual);
 
+List *get_rel_partition_info(RelOptInfo *rel, RangeTblEntry *rte, List *clauses,
+						 RuntimePruningInfo **prtp, bool is_extern);
+
 #ifdef OPTIMIZER_DEBUG
 extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
 #endif
-- 
1.8.3.1

#2Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#1)
1 attachment(s)
Re: Runtime Partition Pruning

Hello all,

Here is the updated patch which is rebased over v10 of Amit Langote's
path towards faster pruning patch [1]/messages/by-id/b8094e71-2c73-ed8e-d8c3-53f232c8c049@lab.ntt.co.jp. It modifies the PartScanKeyInfo
struct to hold expressions which is then evaluated by the executor to
fetch the correct partitions using the function.

The code still chooses the custom plan instead of the generic plan for
the prepared statements. I am working on it. The following output is
after adding a hack in the code forcing selection of generic plan.

postgres=# EXPLAIN EXECUTE prstmt_select(70000);
QUERY PLAN
------------------------------------------------------------------
Append (cost=0.00..1732.25 rows=2 width=8)
-> Seq Scan on tprt_1 (cost=0.00..847.00 rows=16667 width=8)
Filter: ($1 > col1)
-> Seq Scan on tprt_2 (cost=0.00..847.00 rows=16667 width=8)
Filter: ($1 > col1)
(5 rows)

postgres=# EXPLAIN EXECUTE prstmt_select(200000);
QUERY PLAN
------------------------------------------------------------------
Append (cost=0.00..1732.25 rows=3 width=8)
-> Seq Scan on tprt_1 (cost=0.00..847.00 rows=16667 width=8)
Filter: ($1 > col1)
-> Seq Scan on tprt_2 (cost=0.00..847.00 rows=16667 width=8)
Filter: ($1 > col1)
-> Seq Scan on tprt_3 (cost=0.00..38.25 rows=753 width=8)
Filter: ($1 > col1)
(7 rows)

[1]: /messages/by-id/b8094e71-2c73-ed8e-d8c3-53f232c8c049@lab.ntt.co.jp

Tested on commit: 9b9cb3c4534d717c1c95758670198ebbf8a20af2

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning.patchDownload
From 6ee8c7bfe6c879f274e37c60d0e617d26dc01976 Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Thu, 9 Nov 2017 16:19:28 +0530
Subject: [PATCH] Implement runtime partiton pruning

Patch by: Beena Emerson, Dilip Kumar
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c         | 122 +++----------
 src/backend/executor/nodeAppend.c       | 315 ++++++++++++++++++++++++++++++--
 src/backend/nodes/copyfuncs.c           |  27 +++
 src/backend/nodes/readfuncs.c           |   2 +
 src/backend/optimizer/path/allpaths.c   |  72 ++++++--
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  46 +++++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  16 ++
 src/backend/optimizer/util/pathnode.c   |  36 +++-
 src/backend/optimizer/util/relnode.c    |  23 ++-
 src/backend/utils/cache/plancache.c     |   2 +-
 src/include/catalog/partition.h         |   8 +
 src/include/nodes/execnodes.h           |   5 +
 src/include/nodes/nodes.h               |   1 +
 src/include/nodes/plannodes.h           |   4 +
 src/include/nodes/relation.h            |  87 +++++++++
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   2 +-
 src/include/optimizer/paths.h           |   4 +
 21 files changed, 652 insertions(+), 129 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 152ae60..073164d 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -130,84 +130,6 @@ typedef struct
 } PartClause;
 
 /*
- * PartScanKeyInfo
- *		Bounding scan keys to look up a table's partitions obtained from
- *		mutually-ANDed clauses containing partitioning-compatible operators
- */
-typedef struct PartScanKeyInfo
-{
-	/*
-	 * Constants constituting the *whole* partition key compared using
-	 * partitioning-compatible equality operator(s).  When n_eqkeys > 0, other
-	 * keys (minkeys and maxkeys) are irrelevant.
-	 */
-	Datum	eqkeys[PARTITION_MAX_KEYS];
-	int		n_eqkeys;
-
-	/*
-	 * Constants that constitute the lower bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using > or >=
-	 * operator compatible with partitioning, making this the lower bound in
-	 * a range query.
-	 */
-	Datum	minkeys[PARTITION_MAX_KEYS];
-	int		n_minkeys;
-	bool	min_incl;
-
-	/*
-	 * Constants that constitute the upper bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using < or <=
-	 * operator compatible with partitioning, making this the upper bound in
-	 * a range query.
-	 */
-	Datum	maxkeys[PARTITION_MAX_KEYS];
-	int		n_maxkeys;
-	bool	max_incl;
-
-	/*
-	 * Specifies the type of NullTest that was applied to each of the
-	 * partition key columns or -1 if none was applied.  Partitioning handles
-	 * null partition keys specially depending on the partitioning method in
-	 * use, so get_partitions_for_keys can return partitions according to
-	 * the nullness condition for partition keys.
-	 */
-	NullTestType	keynullness[PARTITION_MAX_KEYS];
-} PartScanKeyInfo;
-
- /* A data structure to represent a partition set. */
-typedef struct PartitionSet
-{
-	/*
-	 * If either  empty or all_parts is true, values of the other fields are
-	 * invalid.
-	 */
-	bool	empty;				/* contains no partitions */
-	bool	all_parts;			/* contains all partitions */
-
-	/*
-	 * In the case of range partitioning, min_part_index contains the index of
-	 * the lowest partition contained in the set and max_datum_index that of
-	 * the highest partition (all partitions between these two indexes
-	 * inclusive are part of the set.)  Since other types of partitioning do
-	 * not impose order on the data contained in successive partitions, these
-	 * fields are not set in that case.
-	 */
-	bool	use_range;
-	int		min_part_idx;
-	int		max_part_idx;
-
-	/*
-	 * other_parts contains the indexes of partitions that are not covered by
-	 * the range defined by min/max indexes.  For example, in the case of
-	 * range partitoning, it will include default partition index (if any).
-	 * Also, this is the only way to return list partitions, because list
-	 * partitions do not have the same ordering property as range partitions,
-	 * so it's pointless to use the min/max range method.
-	 */
-	Bitmapset *other_parts;
-} PartitionSet;
-
-/*
  * PartitionBoundCmpArg - Caller-defined argument to be passed to
  *						  partition_bound_cmp()
  *
@@ -277,9 +199,6 @@ static PartitionSet *partset_copy(const PartitionSet *in);
 static PartitionSet *partset_intersect(PartitionSet *a, const PartitionSet *b);
 static PartitionSet *partset_union(PartitionSet *a, const PartitionSet *b);
 static PartitionSet *partset_new(bool empty, bool all_parts);
-static int classify_partition_bounding_keys(Relation relation, List *clauses,
-								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -287,8 +206,6 @@ static bool partition_cmp_args(Oid partopfamily, Oid partopcintype,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
 				   bool *result);
 static bool partkey_datum_from_expr(const Expr *expr, Datum *value);
-static PartitionSet *get_partitions_for_keys(Relation rel,
-						PartScanKeyInfo *keys);
 
 /*
  * RelationBuildPartitionDesc
@@ -1641,9 +1558,11 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 	List *or_clauses;
 	ListCell *lc;
 
+	memset(&keys, 0, sizeof(PartScanKeyInfo));
+
 	nkeys = classify_partition_bounding_keys(relation, clauses,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, false);
 	if (constfalse)
 		/* None of the partitions will satisfy the clauses. */
 		partset = partset_new(true, false);
@@ -1875,10 +1794,10 @@ partset_union(PartitionSet *a, const PartitionSet *b)
  * the responsibility of the caller to process the argument clauses of each of
  * the OR clauses, which would involve recursively calling this function.
  */
-static int
+int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, bool is_runtime)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -2314,7 +2233,6 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_eqkeys = 0;
 
 	/* Populate keys. */
-	memset(keys, 0, sizeof(PartScanKeyInfo));
 	if (n_eqkeys + n_minkeys + n_maxkeys + n_keynullness > 0)
 	{
 		Datum	value;
@@ -2325,7 +2243,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		{
 			if (partkey_datum_from_expr(eqkey_exprs[i], &value))
 			{
-				keys->eqkeys[i] = value;
+				if (value)
+					keys->eqkeys_datums[i] = value;
+				else
+					keys->eqkeys = lappend(keys->eqkeys, eqkey_exprs[i]);
 				n_datums_resolved++;
 			}
 		}
@@ -2337,7 +2258,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		{
 			if (partkey_datum_from_expr(minkey_exprs[i], &value))
 			{
-				keys->minkeys[i] = value;
+				if (value)
+					keys->minkeys_datums[i] = value;
+				else
+					keys->minkeys = lappend(keys->minkeys, minkey_exprs[i]);
 				n_datums_resolved++;
 			}
 		}
@@ -2350,7 +2274,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		{
 			if (partkey_datum_from_expr(maxkey_exprs[i], &value))
 			{
-				keys->maxkeys[i] = value;
+				if (value)
+					keys->maxkeys_datums[i] = value;
+				else
+					keys->maxkeys = lappend(keys->maxkeys, maxkey_exprs[i]);
 				n_datums_resolved++;
 			}
 		}
@@ -2363,6 +2290,9 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_total += n_keynullness;
 	}
 
+	if (n_total == 0)
+		keys = NULL;
+
 	return n_total;
 }
 
@@ -2383,6 +2313,10 @@ partkey_datum_from_expr(const Expr *expr, Datum *value)
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			*value = '\0';
+			return true;
+
 		default:
 			return false;
 	}
@@ -2593,7 +2527,7 @@ partition_cmp_args(Oid partopfamily, Oid partopcintype,
  * Outputs:
  *	Partition set satisfying the keys.
  */
-static PartitionSet *
+PartitionSet *
 get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 {
 	PartitionSet   *partset;
@@ -2677,7 +2611,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_eqkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->eqkeys;
+		arg.datums = keys->eqkeys_datums;
 		arg.ndatums = keys->n_eqkeys;
 		eqoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
@@ -2729,7 +2663,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_minkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->minkeys;
+		arg.datums = keys->minkeys_datums;
 		arg.ndatums = keys->n_minkeys;
 		minoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
@@ -2815,7 +2749,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_maxkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->maxkeys;
+		arg.datums = keys->maxkeys_datums;
 		arg.ndatums = keys->n_maxkeys;
 		maxoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..0aebc90 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -1,4 +1,4 @@
-/*-------------------------------------------------------------------------
+ /*-------------------------------------------------------------------------
  *
  * nodeAppend.c
  *	  routines to handle append nodes.
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "nodes/relation.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "optimizer/clauses.h"
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool exec_append_initialize_next(AppendState *appendstate);
@@ -108,6 +110,109 @@ exec_append_initialize_next(AppendState *appendstate)
 }
 
 /* ----------------------------------------------------------------
+ *	   extern_eval_const_expressions
+ *
+ *		Evalute the expressions to a constant.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo*
+extern_eval_const_expressions(EState *estate, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	int			i;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	i = 0;
+	foreach(lc, old_keys->eqkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n = eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		keys->eqkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	i = 0;
+	foreach(lc, old_keys->minkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n = eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		keys->minkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	i = 0;
+	foreach(lc, old_keys->maxkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n = eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		keys->maxkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+
+	return keys;
+}
+
+
+/* ----------------------------------------------------------------
+ *	   exec_InitExpr
+ *
+ *		Evaluate and store the ooutput of ExecInitExpr for each of the keys.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo *
+exec_InitExpr(PlanState *parent, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+
+	foreach(lc, old_keys->eqkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->eqkeys = lappend(keys->eqkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	foreach(lc, old_keys->minkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->minkeys = lappend(keys->minkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	foreach(lc, old_keys->maxkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->maxkeys = lappend(keys->maxkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
+/* ----------------------------------------------------------------
  *		ExecInitAppend
  *
  *		Begin all of the subscans of the append node.
@@ -165,22 +270,97 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
-	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
-	 */
-	i = 0;
-	foreach(lc, node->appendplans)
+	if (node->et_keys)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		List	   *temp_appendplans = NIL;
+		Append	   *temp_node;
+		Relation	rel;
+		PartScanKeyInfo *keys;
+		PartitionSet *partset;
+		int			i;
+
+		keys = extern_eval_const_expressions(estate, node->et_keys);
+
+		rel = relation_open(node->parentoid, NoLock);
+		partset = get_partitions_for_keys(rel, keys);
+		relation_close(rel, NoLock);
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
-		i++;
+		if (!partset->empty)
+		{
+			if (partset->all_parts)
+			{
+				i = 0;
+				foreach(lc, node->appendplans)
+				{
+					Plan	   *initNode = (Plan *) lfirst(lc);
+
+					temp_appendplans = lappend(temp_appendplans, initNode);
+
+					appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+					i++;
+				}
+			}
+			else
+			{
+				int			j;
+
+				i = 0;
+				if (partset->use_range)
+				{
+					for (j = partset->min_part_idx; j <= partset->max_part_idx; j++)
+					{
+						Plan	   *initNode = (Plan *) list_nth(node->appendplans, j);
+
+						temp_appendplans = lappend(temp_appendplans, initNode);
+						appendplanstates[i++] = ExecInitNode(initNode, estate, eflags);
+					}
+				}
+
+				if (!bms_is_empty(partset->other_parts))
+				{
+					while ((j = bms_first_member(partset->other_parts)) >= 0)
+					{
+						Plan	   *initNode = (Plan *) list_nth(node->appendplans, j);
+
+						temp_appendplans = lappend(temp_appendplans, initNode);
+						appendplanstates[i++] = ExecInitNode(initNode, estate, eflags);
+					}
+				}
+			}
+		}
+		/* create new AppendState for our append node */
+		temp_node = copyObject(node);
+		temp_node->appendplans = temp_appendplans;
+		((Plan *) temp_node)->plan_rows = i;
+		appendstate->as_nplans = i;
+		appendstate->ps.plan = (Plan *) temp_node;
+		appendstate->ps.state = estate;
+		appendstate->ps.ExecProcNode = ExecAppend;
+		appendstate->appendplans = (PlanState **) palloc0(i * sizeof(PlanState *));
+		appendstate->appendplans = appendplanstates;
 	}
+	else
+	{
+		appendstate->parentoid = node->parentoid;
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		/*
+		 * call ExecInitNode on each of the plans to be executed and save the
+		 * results into the array "appendplans".
+		 */
+		i = 0;
+		foreach(lc, node->appendplans)
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+
+			appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			i++;
+		}
+	}
+
+	if (node->ex_keys)
+		appendstate->ex_keys = exec_InitExpr((PlanState *) appendstate, node->ex_keys);
 
-	/*
-	 * initialize output tuple type
-	 */
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
@@ -237,7 +417,21 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->index)
+			{
+				if (++node->as_whichpartition < list_length(node->index))
+					node->as_whichplan = list_nth_int(node->index, node->as_whichpartition);
+
+				return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -275,11 +469,95 @@ ExecEndAppend(AppendState *node)
 		ExecEndNode(appendplans[i]);
 }
 
+/* ----------------------------------------------------------------
+ *	   exec_EvalExpr
+ *
+ *		Convert the expressions to a constant.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo*
+exec_EvalExpr(ExprContext *econtext, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	int			i;
+	bool		isnull;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	i = 0;
+	foreach(lc, old_keys->eqkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->eqkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	i = 0;
+	foreach(lc, old_keys->minkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->minkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	i = 0;
+	foreach(lc, old_keys->maxkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->maxkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
 void
 ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	if (node->ps.chgParam != NULL && node->ex_keys)
+	{
+		PartitionSet *partset;
+
+		node->index = NULL;
+		node->ex_keys = exec_EvalExpr(node->ps.ps_ExprContext, node->ex_keys);
+		if (node->parentoid)
+		{
+			Relation	rel = relation_open(node->parentoid, NoLock);
+
+			partset = get_partitions_for_keys(rel, node->ex_keys);
+			relation_close(rel, NoLock);
+		}
+		if (!partset->empty)
+		{
+			if (!partset->all_parts)
+			{
+				int			j;
+
+				i = 0;
+				if (partset->use_range)
+				{
+					for (j = partset->min_part_idx; j <= partset->max_part_idx; j++)
+						node->index = lappend_int(node->index, j);
+				}
+				if (!bms_is_empty(partset->other_parts))
+				{
+					while ((j = bms_first_member(partset->other_parts)) >= 0)
+						node->index = lappend_int(node->index, j);
+				}
+			}
+		}
+	}
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +576,13 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->index)
+	{
+		node->as_whichplan = linitial_int(node->index);
+		node->as_whichpartition = 0;
+	}
+	else
+		node->as_whichplan = 0;
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca..95dddab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -224,6 +224,27 @@ _copyModifyTable(const ModifyTable *from)
 	return newnode;
 }
 
+static PartScanKeyInfo *
+_copyPartScanKeyInfo(const PartScanKeyInfo *from)
+{
+	PartScanKeyInfo *newnode = makeNode(PartScanKeyInfo);
+
+	COPY_SCALAR_FIELD(n_eqkeys);
+	COPY_NODE_FIELD(eqkeys);
+
+	COPY_SCALAR_FIELD(n_minkeys);
+	COPY_NODE_FIELD(minkeys);
+	COPY_SCALAR_FIELD(min_incl);
+
+	COPY_SCALAR_FIELD(n_maxkeys);
+	COPY_NODE_FIELD(maxkeys);
+	COPY_SCALAR_FIELD(max_incl);
+
+	memcpy(newnode->keynullness, from->keynullness, sizeof(newnode->keynullness));
+
+	return newnode;
+}
+
 /*
  * _copyAppend
  */
@@ -242,6 +263,9 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(ex_keys);
+	COPY_NODE_FIELD(et_keys);
+	COPY_SCALAR_FIELD(parentoid);
 
 	return newnode;
 }
@@ -5014,6 +5038,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartScanKeyInfo:
+			retval = _copyPartScanKeyInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index ccb6a1f..15e6764 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2652,6 +2652,8 @@ parseNodeString(void)
 		return_value = _readLimit();
 	else if (MATCH("NESTLOOPPARAM", 13))
 		return_value = _readNestLoopParam();
+	else if (MATCH("PARTSCANKEYINFO", 15))
+		return_value = _readNestLoopParam();
 	else if (MATCH("PLANROWMARK", 11))
 		return_value = _readPlanRowMark();
 	else if (MATCH("PLANINVALITEM", 13))
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 12f409e..2d134c7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,6 +18,7 @@
 #include <limits.h>
 #include <math.h>
 
+#include "catalog/partition.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
 #include "catalog/partition.h"
@@ -139,11 +140,11 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 						List *live_childrels);
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte);
+						  RangeTblEntry *rte, bool is_runtime);
 static List *match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
 						 bool *contains_const,
-						 bool *constfalse);
+						 bool *constfalse, bool is_runtime);
 
 
 /*
@@ -865,6 +866,45 @@ intcmp(const void *va, const void *vb)
 	return (a > b) ? 1 : -1;
 }
 
+PartScanKeyInfo*
+get_append_partition_info(RelOptInfo *rel, RangeTblEntry *rte,
+						  bool is_runtime, List *clauses)
+{
+	bool		constfalse,
+				contains_constant;
+	PartScanKeyInfo *keys = NULL;
+	List	   *partclauses;
+
+	/*
+	 * Get the clauses that match the partition key, including information
+	 * about any nullness tests against partition keys.  Set keynullness to a
+	 * invalid value of NullTestType, which 0 is not.
+	 */
+	partclauses = match_clauses_to_partkey(rel,
+										   list_copy(clauses),
+										   &contains_constant,
+										   &constfalse, is_runtime);
+
+	if (partclauses != NIL)
+	{
+		bool		constfalse;
+		List	   *or_clauses;
+		Relation	parent = heap_open(rte->relid, NoLock);
+		PartScanKeyInfo *temp_keys = makeNode(PartScanKeyInfo);
+		int			nkeys = classify_partition_bounding_keys(parent, partclauses,
+															 temp_keys,
+															 &constfalse,
+															 &or_clauses,
+															 true);
+
+		if (nkeys > 0)
+			keys = temp_keys;
+
+		heap_close(parent, NoLock);
+	}
+	return keys;
+}
+
 /*
  * get_append_rel_partitions
  *		Return the list of partitions of rel that pass the clauses mentioned
@@ -875,7 +915,7 @@ intcmp(const void *va, const void *vb)
 static List *
 get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte)
+						  RangeTblEntry *rte, bool is_runtime)
 {
 	Relation	parent = heap_open(rte->relid, NoLock);
 	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
@@ -898,7 +938,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	partclauses = match_clauses_to_partkey(rel,
 										   list_copy(rel->baserestrictinfo),
 										   &contains_const,
-										   &constfalse);
+										   &constfalse, is_runtime);
 
 	/*
 	 * If the matched clauses contains at least some constant operands, use
@@ -1004,7 +1044,8 @@ static List *
 match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
 						 bool *contains_const,
-						 bool *constfalse)
+						 bool *constfalse,
+						 bool is_runtime)
 {
 	PartitionScheme	partscheme = rel->part_scheme;
 	List	   *result = NIL;
@@ -1051,7 +1092,8 @@ match_clauses_to_partkey(RelOptInfo *rel,
 				match_clauses_to_partkey(rel,
 									 list_copy(((BoolExpr *) clause)->args),
 										 &contains_const1,
-										 &constfalse1) != NIL)
+										 &constfalse1,
+										 is_runtime) != NIL)
 			{
 				result = lappend(result, clause);
 				*contains_const = contains_const1;
@@ -1123,6 +1165,12 @@ match_clauses_to_partkey(RelOptInfo *rel,
 					/* Neither argument matches the partition key. */
 					continue;
 
+				if (IsA(constexpr, Const) &&is_runtime)
+					continue;
+
+				if (IsA(constexpr, Param) &&!is_runtime)
+					continue;
+
 				/*
 				 * Only allow strict operators to think sanely about the
 				 * behavior with null arguments.
@@ -1228,7 +1276,7 @@ match_clauses_to_partkey(RelOptInfo *rel,
 				if (!PartCollMatchesExprColl(partcoll, saop_coll))
 					continue;
 			}
-			else if (IsA(clause, NullTest))
+			else if (!is_runtime && IsA(clause, NullTest))
 			{
 				NullTest   *nulltest = (NullTest *) clause;
 				Node	   *arg = (Node *) nulltest->arg;
@@ -1288,7 +1336,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	}
 	else
 	{
-		rel_appinfos = get_append_rel_partitions(root, rel, rte);
+		rel_appinfos = get_append_rel_partitions(root, rel, rte, false);
 		rel->live_partitioned_rels = list_make1_int(rti);
 	}
 
@@ -1850,7 +1898,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1877,7 +1925,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1931,7 +1979,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2168,7 +2216,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 7356683..af7e790 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9c74e39..9014ef2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,47 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static PartScanKeyInfo *
+replace_partition_nestloop_params(PlannerInfo *root, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	foreach(lc, old_keys->eqkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->eqkeys = lappend(keys->eqkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	foreach(lc, old_keys->minkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->minkeys = lappend(keys->minkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	foreach(lc, old_keys->maxkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->maxkeys = lappend(keys->maxkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1104,11 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+
+	plan->ex_keys = replace_partition_nestloop_params(root, best_path->ex_keys);
+	plan->et_keys = best_path->et_keys;
+	plan->parentoid = best_path->parentoid;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index fd0e483..8a8876f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3675,7 +3675,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 30cdd3d..8634460 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -5197,3 +5197,19 @@ tlist_matches_coltypelist(List *tlist, List *coltypelist)
 
 	return true;
 }
+
+Node *
+eval_const_expressions_from_list(ParamListInfo prmList, Node *node)
+{
+	eval_const_expressions_context context;
+
+	if (prmList)
+		context.boundParams = prmList;	/* bound Params */
+	else
+		context.boundParams = NULL;
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 36ec025..8910d39 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,40 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	if (root && (required_outer || rel->baserestrictinfo))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+		List	   *query_quals = rel->baserestrictinfo;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				ParamPathInfo *ppi = pathnode->path.param_info;
+
+				/* Set the ex_keys for Exec Params */
+				if (ppi)
+				{
+					List	   *ppi_clauses = ppi->ppi_clauses;
+
+					pathnode->ex_keys =
+						get_append_partition_info(rel, rte, true, ppi_clauses);
+				}
+				/* Set et_keys for extern params */
+				if (query_quals)
+				{
+					pathnode->et_keys =
+						get_append_partition_info(rel, rte, true, query_quals);
+				}
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index b06696b..ca83bb8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1556,6 +1556,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1567,11 +1570,29 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+
+	/*
+	 * Generally for appendrel we don't fetch the clause from the the
+	 * join clause (only we do so for baserel), but for identifying whether
+	 * the appendrel is applicable for runtime pruning or not.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7da99a9..eb54eaf 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -112,4 +113,11 @@ extern List *get_proposed_default_constraint(List *new_part_constaints);
 void get_partitions_from_clauses(Relation relation, List *partclauses,
 								 int *min_part_idx, int *max_part_idx,
 								 Bitmapset **other_parts);
+
+ PartitionSet *get_partitions_for_keys(Relation rel,
+						PartScanKeyInfo *keys);
+
+extern int classify_partition_bounding_keys(Relation relation, List *clauses,
+								 PartScanKeyInfo* keys, bool *constfalse,
+								 List **or_clauses, bool is_runtime);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..60cbb36 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,11 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Oid			parentoid;
+	List       *index;
+
+	int		as_whichpartition;
+	PartScanKeyInfo *ex_keys;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index ffeeb49..5e09258 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -87,6 +87,7 @@ typedef enum NodeTag
 	T_NestLoopParam,
 	T_PlanRowMark,
 	T_PlanInvalItem,
+	T_PartScanKeyInfo,
 
 	/*
 	 * TAGS FOR PLAN STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index dd74efa..96f0632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,9 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+	Oid			parentoid;
+	PartScanKeyInfo *ex_keys;
+	PartScanKeyInfo *et_keys;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9c67bd1..ee4f6de 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -363,6 +363,89 @@ typedef struct PartitionSchemeData
 
 typedef struct PartitionSchemeData *PartitionScheme;
 
+/*
+ * PartScanKeyInfo
+ *		Bounding scan keys to look up a table's partitions obtained from
+ *		mutually-ANDed clauses containing partitioning-compatible operators
+ */
+typedef struct PartScanKeyInfo
+{
+	NodeTag		type;
+
+	/*
+	 * Expressions or constants constituting the *whole* partition key
+	 * compared using partitioning-compatible equality operator(s).  When
+	 * n_eqkeys > 0, other keys (minkeys and maxkeys) are irrelevant.
+	 */
+	List	   *eqkeys;
+	Datum		eqkeys_datums[PARTITION_MAX_KEYS];
+	int			n_eqkeys;
+
+	/*
+	 * Expressions or constants that constitute the lower bound on the
+	 * partition key or a prefix thereof.  The last of those constants is
+	 * compared using > or >= operator compatible with partitioning, making
+	 * this the lower bound in a range query.
+	 */
+	List	   *minkeys;
+	Datum		minkeys_datums[PARTITION_MAX_KEYS];
+	int			n_minkeys;
+	bool		min_incl;
+
+	/*
+	 * Expressions or constants that constitute the upper bound on the
+	 * partition key or a prefix thereof.  The last of those constants is
+	 * compared using < or <= operator compatible with partitioning, making
+	 * this the upper bound in a range query.
+	 */
+	List	   *maxkeys;
+	Datum		maxkeys_datums[PARTITION_MAX_KEYS];
+	int			n_maxkeys;
+	bool		max_incl;
+
+	/*
+	 * Specifies the type of NullTest that was applied to each of the
+	 * partition key columns or -1 if none was applied.  Partitioning handles
+	 * null partition keys specially depending on the partitioning method in
+	 * use, so get_partitions_for_keys can return partitions according to the
+	 * nullness condition for partition keys.
+	 */
+	NullTestType keynullness[PARTITION_MAX_KEYS];
+}			PartScanKeyInfo;
+
+ /* A data structure to represent a partition set. */
+typedef struct PartitionSet
+{
+	/*
+	 * If either  empty or all_parts is true, values of the other fields are
+	 * invalid.
+	 */
+	bool		empty;			/* contains no partitions */
+	bool		all_parts;		/* contains all partitions */
+
+	/*
+	 * In the case of range partitioning, min_part_index contains the index of
+	 * the lowest partition contained in the set and max_datum_index that of
+	 * the highest partition (all partitions between these two indexes
+	 * inclusive are part of the set.)  Since other types of partitioning do
+	 * not impose order on the data contained in successive partitions, these
+	 * fields are not set in that case.
+	 */
+	bool		use_range;
+	int			min_part_idx;
+	int			max_part_idx;
+
+	/*
+	 * other_parts contains the indexes of partitions that are not covered by
+	 * the range defined by min/max indexes.  For example, in the case of
+	 * range partitoning, it will include default partition index (if any).
+	 * Also, this is the only way to return list partitions, because list
+	 * partitions do not have the same ordering property as range partitions,
+	 * so it's pointless to use the min/max range method.
+	 */
+	Bitmapset  *other_parts;
+}			PartitionSet;
+
 /*----------
  * RelOptInfo
  *		Per-relation information for planning/optimization
@@ -1289,6 +1372,9 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+	PartScanKeyInfo *ex_keys;	/* Runtime Partition Pruning - EXEC Params */
+	PartScanKeyInfo *et_keys;	/* Runtime Partition Pruning - EXTERN Params */
+	Oid			parentoid;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
@@ -2334,4 +2420,5 @@ typedef struct JoinCostWorkspace
 	int			numbatches;
 } JoinCostWorkspace;
 
+
 #endif							/* RELATION_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index e367221..c19ee31 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prm_list, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..d9b8dff 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,7 +63,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths,
 				   Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..e780b20 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -234,4 +234,8 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern PartScanKeyInfo *get_append_partition_info(
+						  RelOptInfo *rel,
+						  RangeTblEntry *rte, bool is_runtime, List *clauses);
+
 #endif							/* PATHS_H */
-- 
1.8.3.1

#3Robert Haas
robertmhaas@gmail.com
In reply to: Beena Emerson (#2)
Re: Runtime Partition Pruning

On Thu, Nov 9, 2017 at 6:18 AM, Beena Emerson <memissemerson@gmail.com> wrote:

The code still chooses the custom plan instead of the generic plan for
the prepared statements. I am working on it.

I don't think it's really the job of this patch to do anything about
that problem.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#4Amit Kapila
amit.kapila16@gmail.com
In reply to: Robert Haas (#3)
Re: Runtime Partition Pruning

On Thu, Nov 9, 2017 at 9:01 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Nov 9, 2017 at 6:18 AM, Beena Emerson <memissemerson@gmail.com> wrote:

The code still chooses the custom plan instead of the generic plan for
the prepared statements. I am working on it.

I don't think it's really the job of this patch to do anything about
that problem.

+1. I think if we really want to do something about plan choice when
partitions are involved that should be done as a separate patch.

--
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#5amul sul
sulamul@gmail.com
In reply to: Beena Emerson (#2)
Re: Runtime Partition Pruning

On Thu, Nov 9, 2017 at 4:48 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello all,

Here is the updated patch which is rebased over v10 of Amit Langote's
path towards faster pruning patch [1]. It modifies the PartScanKeyInfo
struct to hold expressions which is then evaluated by the executor to
fetch the correct partitions using the function.

Hi Beena,

I have started looking into your patch, here few initial comments
for your 0001 patch:

1.
351 + * Evaluate and store the ooutput of ExecInitExpr for each
of the keys.

Typo: ooutput

2.
822 + if (IsA(constexpr, Const) &&is_runtime)
823 + continue;
824 +
825 + if (IsA(constexpr, Param) &&!is_runtime)
826 + continue;
827 +

Add space after '&&'

3.
1095 + * Generally for appendrel we don't fetch the clause from the the

Typo: Double 'the'

4.
272 -/*-------------------------------------------------------------------------
273 + /*-------------------------------------------------------------------------

Unnecessary hunk.

5.
313 + Node *n =
eval_const_expressions_from_list(estate->es_param_list_info, val);
314 +

Crossing 80 column window. Same at line # 323 & 325

6.
315 + keys->eqkeys_datums[i++] = ((Const *) n)->constvalue;

Don’t we need a check for IsA(n, Const) or assert ?

7.
1011 + if (prmList)
1012 + context.boundParams = prmList; /* bound Params */
1013 + else
1014 + context.boundParams = NULL;

No need of prmList null check, context.boundParams = prmList; is enough.

8. It would be nice if you create a separate patch where you are moving
PartScanKeyInfo and exporting function declaration.

9. Could you please add few regression tests, that would help in
review & testing.

10. Could you please rebase your patch against latest "path toward faster
partition pruning" patch by Amit.

Regards,
Amul

--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers

#6Beena Emerson
memissemerson@gmail.com
In reply to: amul sul (#5)
Re: [HACKERS] Runtime Partition Pruning

Hello Amul,

Thank you for reviewing.

On Fri, Nov 10, 2017 at 4:33 PM, amul sul <sulamul@gmail.com> wrote:

On Thu, Nov 9, 2017 at 4:48 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello all,

Here is the updated patch which is rebased over v10 of Amit Langote's
path towards faster pruning patch [1]. It modifies the PartScanKeyInfo
struct to hold expressions which is then evaluated by the executor to
fetch the correct partitions using the function.

Hi Beena,

I have started looking into your patch, here few initial comments
for your 0001 patch:

1.
351 + * Evaluate and store the ooutput of ExecInitExpr for each
of the keys.

Typo: ooutput

Corrected.

2.
822 + if (IsA(constexpr, Const) &&is_runtime)
823 + continue;
824 +
825 + if (IsA(constexpr, Param) &&!is_runtime)
826 + continue;
827 +

Add space after '&&'

Done.

3.
1095 + * Generally for appendrel we don't fetch the clause from the the

Typo: Double 'the'

4.
272 -/*-------------------------------------------------------------------------
273 + /*-------------------------------------------------------------------------

Unnecessary hunk.

Removed.

5.
313 + Node *n =
eval_const_expressions_from_list(estate->es_param_list_info, val);
314 +

Crossing 80 column window. Same at line # 323 & 325

Fixed.

6.
315 + keys->eqkeys_datums[i++] = ((Const *) n)->constvalue;

Don’t we need a check for IsA(n, Const) or assert ?

added

7.
1011 + if (prmList)
1012 + context.boundParams = prmList; /* bound Params */
1013 + else
1014 + context.boundParams = NULL;

No need of prmList null check, context.boundParams = prmList; is enough.

8. It would be nice if you create a separate patch where you are moving
PartScanKeyInfo and exporting function declaration.

This is in 0001.

9. Could you please add few regression tests, that would help in
review & testing.

I will make a seperate regression patch and submit soon.

10. Could you please rebase your patch against latest "path toward faster
partition pruning" patch by Amit.

The following is rebased over v11 Amit's patch [1]/messages/by-id/62d21a7b-fea9-f2d7-c33a-8caa12eca612@lab.ntt.co.jp

[1]: /messages/by-id/62d21a7b-fea9-f2d7-c33a-8caa12eca612@lab.ntt.co.jp

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#7Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#6)
2 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

PFA the updated patches.

On Tue, Nov 14, 2017 at 11:45 AM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello Amul,

Thank you for reviewing.

On Fri, Nov 10, 2017 at 4:33 PM, amul sul <sulamul@gmail.com> wrote:

On Thu, Nov 9, 2017 at 4:48 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello all,

Here is the updated patch which is rebased over v10 of Amit Langote's
path towards faster pruning patch [1]. It modifies the PartScanKeyInfo
struct to hold expressions which is then evaluated by the executor to
fetch the correct partitions using the function.

Hi Beena,

I have started looking into your patch, here few initial comments
for your 0001 patch:

1.
351 + * Evaluate and store the ooutput of ExecInitExpr for each
of the keys.

Typo: ooutput

Corrected.

2.
822 + if (IsA(constexpr, Const) &&is_runtime)
823 + continue;
824 +
825 + if (IsA(constexpr, Param) &&!is_runtime)
826 + continue;
827 +

Add space after '&&'

Done.

3.
1095 + * Generally for appendrel we don't fetch the clause from the the

Typo: Double 'the'

4.
272 -/*-------------------------------------------------------------------------
273 + /*-------------------------------------------------------------------------

Unnecessary hunk.

Removed.

5.
313 + Node *n =
eval_const_expressions_from_list(estate->es_param_list_info, val);
314 +

Crossing 80 column window. Same at line # 323 & 325

Fixed.

6.
315 + keys->eqkeys_datums[i++] = ((Const *) n)->constvalue;

Don’t we need a check for IsA(n, Const) or assert ?

added

7.
1011 + if (prmList)
1012 + context.boundParams = prmList; /* bound Params */
1013 + else
1014 + context.boundParams = NULL;

No need of prmList null check, context.boundParams = prmList; is enough.

8. It would be nice if you create a separate patch where you are moving
PartScanKeyInfo and exporting function declaration.

This is in 0001.

9. Could you please add few regression tests, that would help in
review & testing.

I will make a seperate regression patch and submit soon.

10. Could you please rebase your patch against latest "path toward faster
partition pruning" patch by Amit.

The following is rebased over v11 Amit's patch [1]

[1] /messages/by-id/62d21a7b-fea9-f2d7-c33a-8caa12eca612@lab.ntt.co.jp

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0002-Implement-runtime-partiton-pruning_v2.patchapplication/octet-stream; name=0002-Implement-runtime-partiton-pruning_v2.patchDownload
From 77e135e68ea1753720921e4805b773fef2259fdc Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Tue, 14 Nov 2017 11:19:33 +0530
Subject: [PATCH 2/2] Implement runtime partiton pruning

Patch by: Beena Emerson, Dilip Kumar
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c         |  53 +++---
 src/backend/executor/nodeAppend.c       | 274 ++++++++++++++++++++++++++++++--
 src/backend/nodes/copyfuncs.c           |  27 ++++
 src/backend/nodes/readfuncs.c           |   2 +
 src/backend/optimizer/path/allpaths.c   |  76 +++++++--
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  46 ++++++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  16 ++
 src/backend/optimizer/util/pathnode.c   |  36 ++++-
 src/backend/optimizer/util/relnode.c    |  23 ++-
 src/include/catalog/partition.h         |   2 +-
 src/include/nodes/execnodes.h           |   4 +
 src/include/nodes/nodes.h               |   1 +
 src/include/nodes/plannodes.h           |   4 +
 src/include/nodes/relation.h            |  54 ++++---
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   4 +-
 src/include/optimizer/paths.h           |   5 +
 20 files changed, 556 insertions(+), 80 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 97f3050..7ef97fe 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1529,9 +1529,11 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 	List *or_clauses;
 	ListCell *lc;
 
+	memset(&keys, 0, sizeof(PartScanKeyInfo));
+
 	nkeys = classify_partition_bounding_keys(relation, clauses,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, false);
 	/*
 	 * Only look up in the partition decriptor if the query provides
 	 * constraints on the keys at all.
@@ -1595,7 +1597,7 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, bool is_runtime)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -2079,7 +2081,6 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_eqkeys = 0;
 
 	/* Populate keys. */
-	memset(keys, 0, sizeof(PartScanKeyInfo));
 	if (n_eqkeys + n_minkeys + n_maxkeys + n_keynullness > 0)
 	{
 		Datum	value;
@@ -2088,11 +2089,12 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_eqkeys; i++)
 		{
-			if (partkey_datum_from_expr(eqkey_exprs[i], &value))
-			{
-				keys->eqkeys[i] = value;
-				n_datums_resolved++;
-			}
+			if (IsA(eqkey_exprs[i], Const) &&
+				partkey_datum_from_expr(eqkey_exprs[i], &value))
+				keys->eqkeys_datums[i] = value;
+			else
+				keys->eqkeys = lappend(keys->eqkeys, eqkey_exprs[i]);
+			n_datums_resolved++;
 		}
 		keys->n_eqkeys = n_datums_resolved;
 		n_total += keys->n_eqkeys;
@@ -2100,11 +2102,12 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_minkeys; i++)
 		{
-			if (partkey_datum_from_expr(minkey_exprs[i], &value))
-			{
-				keys->minkeys[i] = value;
-				n_datums_resolved++;
-			}
+			if (IsA(minkey_exprs[i], Const) &&
+				partkey_datum_from_expr(minkey_exprs[i], &value))
+				keys->minkeys_datums[i] = value;
+			else
+				keys->minkeys = lappend(keys->minkeys, minkey_exprs[i]);
+			n_datums_resolved++;
 		}
 		keys->n_minkeys = n_datums_resolved;
 		n_total += keys->n_minkeys;
@@ -2113,11 +2116,12 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_maxkeys; i++)
 		{
-			if (partkey_datum_from_expr(maxkey_exprs[i], &value))
-			{
-				keys->maxkeys[i] = value;
-				n_datums_resolved++;
-			}
+			if (IsA(maxkey_exprs[i], Const) &&
+				partkey_datum_from_expr(maxkey_exprs[i], &value))
+				keys->maxkeys_datums[i] = value;
+			else
+				keys->maxkeys = lappend(keys->maxkeys, maxkey_exprs[i]);
+			n_datums_resolved++;
 		}
 		keys->n_maxkeys = n_datums_resolved;
 		n_total += keys->n_maxkeys;
@@ -2128,6 +2132,9 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_total += n_keynullness;
 	}
 
+	if (n_total == 0)
+		keys = NULL;
+
 	return n_total;
 }
 
@@ -2148,6 +2155,10 @@ partkey_datum_from_expr(const Expr *expr, Datum *value)
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			*value = '\0';
+			return true;
+
 		default:
 			return false;
 	}
@@ -2427,7 +2438,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_eqkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->eqkeys;
+		arg.datums = keys->eqkeys_datums;
 		arg.ndatums = keys->n_eqkeys;
 		eqoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
@@ -2475,7 +2486,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_minkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->minkeys;
+		arg.datums = keys->minkeys_datums;
 		arg.ndatums = keys->n_minkeys;
 		minoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
@@ -2561,7 +2572,7 @@ get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 	if (keys->n_maxkeys > 0)
 	{
 		memset(&arg, 0, sizeof(PartitionBoundCmpArg));
-		arg.datums = keys->maxkeys;
+		arg.datums = keys->maxkeys_datums;
 		arg.ndatums = keys->n_maxkeys;
 		maxoff = partition_bound_bsearch(partkey, boundinfo, &arg, &is_equal);
 
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..c0e0c05 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "nodes/relation.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "optimizer/clauses.h"
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool exec_append_initialize_next(AppendState *appendstate);
@@ -108,6 +110,114 @@ exec_append_initialize_next(AppendState *appendstate)
 }
 
 /* ----------------------------------------------------------------
+ *	   extern_eval_const_expressions
+ *
+ *		Evalute the expressions to a constant.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo *
+extern_eval_const_expressions(EState *estate, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	int			i;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	i = 0;
+	foreach(lc, old_keys->eqkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n =
+		eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		Assert(IsA(n, Const));
+		keys->eqkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	i = 0;
+	foreach(lc, old_keys->minkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n =
+		eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		Assert(IsA(n, Const));
+		keys->minkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	i = 0;
+	foreach(lc, old_keys->maxkeys)
+	{
+		Node	   *val = lfirst(lc);
+		Node	   *n =
+		eval_const_expressions_from_list(estate->es_param_list_info, val);
+
+		Assert(IsA(n, Const));
+		keys->maxkeys_datums[i++] = ((Const *) n)->constvalue;
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+
+	return keys;
+}
+
+/* ----------------------------------------------------------------
+ *	   exec_InitExpr
+ *
+ *		Evaluate and store the output of ExecInitExpr for each of the keys.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo *
+exec_InitExpr(PlanState *parent, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+
+	foreach(lc, old_keys->eqkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->eqkeys = lappend(keys->eqkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	foreach(lc, old_keys->minkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->minkeys = lappend(keys->minkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	foreach(lc, old_keys->maxkeys)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		keys->maxkeys = lappend(keys->maxkeys, ExecInitExpr(val, parent));
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
+/* ----------------------------------------------------------------
  *		ExecInitAppend
  *
  *		Begin all of the subscans of the append node.
@@ -165,22 +272,67 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
-	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
-	 */
-	i = 0;
-	foreach(lc, node->appendplans)
+	if (node->et_keys)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		List	   *temp_appendplans = NIL;
+		Append	   *temp_node;
+		Relation	rel;
+		PartScanKeyInfo *keys;
+		Bitmapset  *partset;
+		int			i,
+					j;
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
-		i++;
+		keys = extern_eval_const_expressions(estate, node->et_keys);
+
+		rel = relation_open(node->parentoid, NoLock);
+		partset = get_partitions_for_keys(rel, keys);
+		relation_close(rel, NoLock);
+
+		if (!bms_is_empty(partset))
+		{
+			i = 0;
+			while ((j = bms_first_member(partset)) >= 0)
+			{
+				Plan	   *initNode = (Plan *) list_nth(node->appendplans, j);
+
+				temp_appendplans = lappend(temp_appendplans, initNode);
+				appendplanstates[i++] = ExecInitNode(initNode, estate, eflags);
+			}
+		}
+		/* create new AppendState for our append node */
+		temp_node = copyObject(node);
+		temp_node->appendplans = temp_appendplans;
+		((Plan *) temp_node)->plan_rows = i;
+		appendstate->as_nplans = i;
+		appendstate->ps.plan = (Plan *) temp_node;
+		appendstate->ps.state = estate;
+		appendstate->ps.ExecProcNode = ExecAppend;
+		appendstate->appendplans = (PlanState **) palloc0(i * sizeof(PlanState *));
+		appendstate->appendplans = appendplanstates;
 	}
+	else
+	{
+		appendstate->parentoid = node->parentoid;
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		/*
+		 * call ExecInitNode on each of the plans to be executed and save the
+		 * results into the array "appendplans".
+		 */
+		i = 0;
+		foreach(lc, node->appendplans)
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+
+			appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			i++;
+		}
+	}
+
+	if (node->ex_keys)
+		appendstate->ex_keys =
+			exec_InitExpr((PlanState *) appendstate, node->ex_keys);
 
-	/*
-	 * initialize output tuple type
-	 */
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
@@ -237,7 +389,21 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->index)
+			{
+				if (++node->as_whichpartition < list_length(node->index))
+					node->as_whichplan = list_nth_int(node->index, node->as_whichpartition);
+
+				return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -275,11 +441,84 @@ ExecEndAppend(AppendState *node)
 		ExecEndNode(appendplans[i]);
 }
 
+/* ----------------------------------------------------------------
+ *	   exec_EvalExpr
+ *
+ *		Convert the expressions to a constant.
+ * ----------------------------------------------------------------
+ */
+static PartScanKeyInfo *
+exec_EvalExpr(ExprContext *econtext, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	int			i;
+	bool		isnull;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	i = 0;
+	foreach(lc, old_keys->eqkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->eqkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	i = 0;
+	foreach(lc, old_keys->minkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->minkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	i = 0;
+	foreach(lc, old_keys->maxkeys)
+	{
+		ExprState  *val = lfirst(lc);
+
+		keys->maxkeys_datums[i++] = ExecEvalExpr(val, econtext, &isnull);
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
 void
 ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	if (node->ps.chgParam != NULL && node->ex_keys)
+	{
+		Bitmapset  *partset;
+		PartScanKeyInfo *ex_keys;
+
+		node->index = NIL;
+		ex_keys = exec_EvalExpr(node->ps.ps_ExprContext, node->ex_keys);
+		if (node->parentoid)
+		{
+			Relation	rel = relation_open(node->parentoid, NoLock);
+
+			partset = get_partitions_for_keys(rel, ex_keys);
+			relation_close(rel, NoLock);
+		}
+		if (!bms_is_empty(partset))
+		{
+			int			j;
+
+			while ((j = bms_first_member(partset)) >= 0)
+				node->index = lappend_int(node->index, j);
+		}
+	}
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +537,13 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->index)
+	{
+		node->as_whichplan = linitial_int(node->index);
+		node->as_whichpartition = 0;
+	}
+	else
+		node->as_whichplan = 0;
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca..95dddab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -224,6 +224,27 @@ _copyModifyTable(const ModifyTable *from)
 	return newnode;
 }
 
+static PartScanKeyInfo *
+_copyPartScanKeyInfo(const PartScanKeyInfo *from)
+{
+	PartScanKeyInfo *newnode = makeNode(PartScanKeyInfo);
+
+	COPY_SCALAR_FIELD(n_eqkeys);
+	COPY_NODE_FIELD(eqkeys);
+
+	COPY_SCALAR_FIELD(n_minkeys);
+	COPY_NODE_FIELD(minkeys);
+	COPY_SCALAR_FIELD(min_incl);
+
+	COPY_SCALAR_FIELD(n_maxkeys);
+	COPY_NODE_FIELD(maxkeys);
+	COPY_SCALAR_FIELD(max_incl);
+
+	memcpy(newnode->keynullness, from->keynullness, sizeof(newnode->keynullness));
+
+	return newnode;
+}
+
 /*
  * _copyAppend
  */
@@ -242,6 +263,9 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(ex_keys);
+	COPY_NODE_FIELD(et_keys);
+	COPY_SCALAR_FIELD(parentoid);
 
 	return newnode;
 }
@@ -5014,6 +5038,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartScanKeyInfo:
+			retval = _copyPartScanKeyInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index ccb6a1f..15e6764 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -2652,6 +2652,8 @@ parseNodeString(void)
 		return_value = _readLimit();
 	else if (MATCH("NESTLOOPPARAM", 13))
 		return_value = _readNestLoopParam();
+	else if (MATCH("PARTSCANKEYINFO", 15))
+		return_value = _readNestLoopParam();
 	else if (MATCH("PLANROWMARK", 11))
 		return_value = _readPlanRowMark();
 	else if (MATCH("PLANINVALITEM", 13))
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a3d0468..166eb2e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,6 +18,7 @@
 #include <limits.h>
 #include <math.h>
 
+#include "catalog/partition.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
 #include "catalog/partition.h"
@@ -140,11 +141,11 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 						List *live_childrels);
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte);
+						  RangeTblEntry *rte, bool is_runtime);
 static List *match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
 						 bool *contains_const,
-						 bool *constfalse);
+						 bool *constfalse, bool is_runtime);
 
 
 /*
@@ -855,6 +856,47 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 	rel->fdwroutine->GetForeignPaths(root, rel, rte->relid);
 }
 
+
+PartScanKeyInfo *
+get_append_partition_info(RelOptInfo *rel, RangeTblEntry *rte,
+						  bool is_runtime, List *clauses)
+{
+	bool		constfalse,
+				contains_constant;
+	PartScanKeyInfo *keys = NULL;
+	List	   *partclauses;
+
+	/*
+	 * Get the clauses that match the partition key, including information
+	 * about any nullness tests against partition keys.  Set keynullness to a
+	 * invalid value of NullTestType, which 0 is not.
+	 */
+	partclauses = match_clauses_to_partkey(rel,
+										   list_copy(clauses),
+										   &contains_constant,
+										   &constfalse, is_runtime);
+
+	if (partclauses != NIL)
+	{
+		bool		constfalse;
+		List	   *or_clauses;
+		Relation	parent = heap_open(rte->relid, NoLock);
+		PartScanKeyInfo *temp_keys = makeNode(PartScanKeyInfo);
+		int			nkeys = classify_partition_bounding_keys(parent, partclauses,
+															 temp_keys,
+															 &constfalse,
+															 &or_clauses,
+															 true);
+
+		if (nkeys > 0)
+			keys = temp_keys;
+
+		heap_close(parent, NoLock);
+	}
+	return keys;
+}
+
+
 /*
  * get_append_rel_partitions
  *		Return the list of partitions of rel that pass the clauses mentioned
@@ -865,7 +907,7 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 static List *
 get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte)
+						  RangeTblEntry *rte, bool is_runtime)
 {
 	Relation	parent = heap_open(rte->relid, NoLock);
 	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
@@ -884,7 +926,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	partclauses = match_clauses_to_partkey(rel,
 										   list_copy(rel->baserestrictinfo),
 										   &contains_const,
-										   &constfalse);
+										   &constfalse, is_runtime);
 
 	/*
 	 * If the matched clauses contains at least some constant operands, use
@@ -956,7 +998,8 @@ static List *
 match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
 						 bool *contains_const,
-						 bool *constfalse)
+						 bool *constfalse,
+						 bool is_runtime)
 {
 	PartitionScheme	partscheme = rel->part_scheme;
 	List	   *result = NIL;
@@ -1003,7 +1046,8 @@ match_clauses_to_partkey(RelOptInfo *rel,
 				match_clauses_to_partkey(rel,
 									 list_copy(((BoolExpr *) clause)->args),
 										 &contains_const1,
-										 &constfalse1) != NIL)
+										 &constfalse1,
+										 is_runtime) != NIL)
 			{
 				result = lappend(result, clause);
 				*contains_const = contains_const1;
@@ -1079,6 +1123,12 @@ match_clauses_to_partkey(RelOptInfo *rel,
 					/* Neither argument matches the partition key. */
 					continue;
 
+				if (IsA(constexpr, Const) && is_runtime)
+					continue;
+
+				if (IsA(constexpr, Param) && !is_runtime)
+					continue;
+
 				/*
 				 * Only allow strict operators to think sanely about the
 				 * behavior with null arguments.
@@ -1184,7 +1234,7 @@ match_clauses_to_partkey(RelOptInfo *rel,
 				if (!PartCollMatchesExprColl(partcoll, saop_coll))
 					continue;
 			}
-			else if (IsA(clause, NullTest))
+			else if (!is_runtime && IsA(clause, NullTest))
 			{
 				NullTest   *nulltest = (NullTest *) clause;
 				Node	   *arg = (Node *) nulltest->arg;
@@ -1200,7 +1250,7 @@ match_clauses_to_partkey(RelOptInfo *rel,
 			 * Boolean conditions have a special shape, which accept if the
 			 * partitioning opfamily accepts Boolean conditions.
 			 */
-			else if (IsBooleanOpfamily(partopfamily) &&
+			else if (!is_runtime && IsBooleanOpfamily(partopfamily) &&
 					 (IsA(clause, BooleanTest) ||
 					  IsA(clause, Var) || not_clause((Node *) clause)))
 			{
@@ -1272,7 +1322,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	}
 	else
 	{
-		rel_appinfos = get_append_rel_partitions(root, rel, rte);
+		rel_appinfos = get_append_rel_partitions(root, rel, rte, false);
 		rel->live_partitioned_rels = list_make1_int(rti);
 	}
 
@@ -1834,7 +1884,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1861,7 +1911,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1915,7 +1965,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2152,7 +2202,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 7356683..af7e790 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9c74e39..9014ef2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,47 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static PartScanKeyInfo *
+replace_partition_nestloop_params(PlannerInfo *root, PartScanKeyInfo *old_keys)
+{
+	PartScanKeyInfo *keys;
+	ListCell   *lc;
+
+	if (old_keys == NULL)
+		return NULL;
+
+	keys = makeNode(PartScanKeyInfo);
+	foreach(lc, old_keys->eqkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->eqkeys = lappend(keys->eqkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_eqkeys = old_keys->n_eqkeys;
+
+	foreach(lc, old_keys->minkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->minkeys = lappend(keys->minkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_minkeys = old_keys->n_minkeys;
+	keys->min_incl = old_keys->min_incl;
+
+	foreach(lc, old_keys->maxkeys)
+	{
+		Node	   *n = lfirst(lc);
+
+		keys->maxkeys = lappend(keys->maxkeys, replace_nestloop_params(root, n));
+	}
+	keys->n_maxkeys = old_keys->n_maxkeys;
+	keys->max_incl = old_keys->max_incl;
+
+	memcpy(keys->keynullness, old_keys->keynullness, sizeof(keys->keynullness));
+	return keys;
+}
+
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1104,11 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+
+	plan->ex_keys = replace_partition_nestloop_params(root, best_path->ex_keys);
+	plan->et_keys = best_path->et_keys;
+	plan->parentoid = best_path->parentoid;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index fd0e483..8a8876f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3675,7 +3675,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 30cdd3d..5479a59f 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -5197,3 +5197,19 @@ tlist_matches_coltypelist(List *tlist, List *coltypelist)
 
 	return true;
 }
+
+Node *
+eval_const_expressions_from_list(ParamListInfo prmList, Node *node)
+{
+	eval_const_expressions_context context;
+
+	if (prmList)
+		context.boundParams = prmList;	/* bound Params */
+	else
+		context.boundParams = NULL;
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 36ec025..8910d39 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,40 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	if (root && (required_outer || rel->baserestrictinfo))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+		List	   *query_quals = rel->baserestrictinfo;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				ParamPathInfo *ppi = pathnode->path.param_info;
+
+				/* Set the ex_keys for Exec Params */
+				if (ppi)
+				{
+					List	   *ppi_clauses = ppi->ppi_clauses;
+
+					pathnode->ex_keys =
+						get_append_partition_info(rel, rte, true, ppi_clauses);
+				}
+				/* Set et_keys for extern params */
+				if (query_quals)
+				{
+					pathnode->et_keys =
+						get_append_partition_info(rel, rte, true, query_quals);
+				}
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index b06696b..9abc79c 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1556,6 +1556,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1567,11 +1570,29 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+
+	/*
+	 * Generally for appendrel we don't fetch the clause from the join clause
+	 * (only we do so for baserel), but for identifying whether the appendrel
+	 * is applicable for runtime pruning or not.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7956f04..4331695 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -114,7 +114,7 @@ extern Bitmapset *get_partitions_from_clauses(Relation relation,
 							List *partclauses);
 extern int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses, bool is_runtime);
 Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..adbac23 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,10 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Oid			parentoid;
+	List	   *index;			/* subplan indexes to scan for runtime pruning */
+	int			as_whichpartition;	/* current partition scanned from list */
+	PartScanKeyInfo *ex_keys;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index ffeeb49..5e09258 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -87,6 +87,7 @@ typedef enum NodeTag
 	T_NestLoopParam,
 	T_PlanRowMark,
 	T_PlanInvalItem,
+	T_PartScanKeyInfo,
 
 	/*
 	 * TAGS FOR PLAN STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index dd74efa..96f0632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,9 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+	Oid			parentoid;
+	PartScanKeyInfo *ex_keys;
+	PartScanKeyInfo *et_keys;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3e47de7..78c7654 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -370,43 +370,48 @@ typedef struct PartitionSchemeData *PartitionScheme;
  */
 typedef struct PartScanKeyInfo
 {
+	NodeTag		type;
+
 	/*
-	 * Constants constituting the *whole* partition key compared using
-	 * partitioning-compatible equality operator(s).  When n_eqkeys > 0, other
-	 * keys (minkeys and maxkeys) are irrelevant.
+	 * Expressions or constants constituting the *whole* partition key
+	 * compared using partitioning-compatible equality operator(s).  When
+	 * n_eqkeys > 0, other keys (minkeys and maxkeys) are irrelevant.
 	 */
-	Datum	eqkeys[PARTITION_MAX_KEYS];
-	int		n_eqkeys;
+	List	   *eqkeys;
+	Datum		eqkeys_datums[PARTITION_MAX_KEYS];
+	int			n_eqkeys;
 
 	/*
-	 * Constants that constitute the lower bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using > or >=
-	 * operator compatible with partitioning, making this the lower bound in
-	 * a range query.
+	 * Expressions or constants that constitute the lower bound on the
+	 * partition key or a prefix thereof.  The last of those constants is
+	 * compared using > or >= operator compatible with partitioning, making
+	 * this the lower bound in a range query.
 	 */
-	Datum	minkeys[PARTITION_MAX_KEYS];
-	int		n_minkeys;
-	bool	min_incl;
+	List	   *minkeys;
+	Datum		minkeys_datums[PARTITION_MAX_KEYS];
+	int			n_minkeys;
+	bool		min_incl;
 
 	/*
-	 * Constants that constitute the upper bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using < or <=
-	 * operator compatible with partitioning, making this the upper bound in
-	 * a range query.
+	 * Expressions or constants that constitute the upper bound on the
+	 * partition key or a prefix thereof.  The last of those constants is
+	 * compared using < or <= operator compatible with partitioning, making
+	 * this the upper bound in a range query.
 	 */
-	Datum	maxkeys[PARTITION_MAX_KEYS];
-	int		n_maxkeys;
-	bool	max_incl;
+	List	   *maxkeys;
+	Datum		maxkeys_datums[PARTITION_MAX_KEYS];
+	int			n_maxkeys;
+	bool		max_incl;
 
 	/*
 	 * Specifies the type of NullTest that was applied to each of the
 	 * partition key columns or -1 if none was applied.  Partitioning handles
 	 * null partition keys specially depending on the partitioning method in
-	 * use, so get_partitions_for_keys can return partitions according to
-	 * the nullness condition for partition keys.
+	 * use, so get_partitions_for_keys can return partitions according to the
+	 * nullness condition for partition keys.
 	 */
-	NullTestType	keynullness[PARTITION_MAX_KEYS];
-} PartScanKeyInfo;
+	NullTestType keynullness[PARTITION_MAX_KEYS];
+}			PartScanKeyInfo;
 
 /*----------
  * RelOptInfo
@@ -1334,6 +1339,9 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+	PartScanKeyInfo *ex_keys;	/* Runtime Partition Pruning - EXEC Params */
+	PartScanKeyInfo *et_keys;	/* Runtime Partition Pruning - EXTERN Params */
+	Oid			parentoid;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index e367221..c7f5262 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prm_list, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..7c1fbfd 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,8 +63,8 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
+				   List *subpaths, Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..525c9df 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -234,4 +234,9 @@ extern PathKey *make_canonical_pathkey(PlannerInfo *root,
 					   EquivalenceClass *eclass, Oid opfamily,
 					   int strategy, bool nulls_first);
 
+extern PartScanKeyInfo *get_append_partition_info(RelOptInfo *rel,
+												   RangeTblEntry *rte,
+												   bool is_runtime,
+												   List *clauses);
+
 #endif							/* PATHS_H */
-- 
1.8.3.1

0001-Refactor-functions-and-structs-required-for-runtime_v2.patchapplication/octet-stream; name=0001-Refactor-functions-and-structs-required-for-runtime_v2.patchDownload
From bad467cbcfa3d8c274f37dcd461ed52fcf619ffd Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Tue, 14 Nov 2017 00:12:46 +0530
Subject: [PATCH 1/2] Refactor functions and structs required for runtime
 pruning

Patch by: Beena Emerson
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c     | 54 ++-----------------------------------
 src/backend/utils/cache/plancache.c |  2 +-
 src/include/catalog/partition.h     |  7 +++++
 src/include/nodes/relation.h        | 45 +++++++++++++++++++++++++++++++
 4 files changed, 55 insertions(+), 53 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index c58c735..97f3050 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -132,51 +132,6 @@ typedef struct
 } PartClause;
 
 /*
- * PartScanKeyInfo
- *		Bounding scan keys to look up a table's partitions obtained from
- *		mutually-ANDed clauses containing partitioning-compatible operators
- */
-typedef struct PartScanKeyInfo
-{
-	/*
-	 * Constants constituting the *whole* partition key compared using
-	 * partitioning-compatible equality operator(s).  When n_eqkeys > 0, other
-	 * keys (minkeys and maxkeys) are irrelevant.
-	 */
-	Datum	eqkeys[PARTITION_MAX_KEYS];
-	int		n_eqkeys;
-
-	/*
-	 * Constants that constitute the lower bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using > or >=
-	 * operator compatible with partitioning, making this the lower bound in
-	 * a range query.
-	 */
-	Datum	minkeys[PARTITION_MAX_KEYS];
-	int		n_minkeys;
-	bool	min_incl;
-
-	/*
-	 * Constants that constitute the upper bound on the partition key or a
-	 * prefix thereof.  The last of those constants is compared using < or <=
-	 * operator compatible with partitioning, making this the upper bound in
-	 * a range query.
-	 */
-	Datum	maxkeys[PARTITION_MAX_KEYS];
-	int		n_maxkeys;
-	bool	max_incl;
-
-	/*
-	 * Specifies the type of NullTest that was applied to each of the
-	 * partition key columns or -1 if none was applied.  Partitioning handles
-	 * null partition keys specially depending on the partitioning method in
-	 * use, so get_partitions_for_keys can return partitions according to
-	 * the nullness condition for partition keys.
-	 */
-	NullTestType	keynullness[PARTITION_MAX_KEYS];
-} PartScanKeyInfo;
-
-/*
  * PartitionBoundCmpArg - Caller-defined argument to be passed to
  *						  partition_bound_cmp()
  *
@@ -242,9 +197,6 @@ static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 
 static Bitmapset *get_partitions_from_clauses_guts(Relation relation,
 								List *clauses);
-static int classify_partition_bounding_keys(Relation relation, List *clauses,
-								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -252,8 +204,6 @@ static bool partition_cmp_args(Oid partopfamily, Oid partopcintype,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
 				   bool *result);
 static bool partkey_datum_from_expr(const Expr *expr, Datum *value);
-static Bitmapset *get_partitions_for_keys(Relation rel,
-						PartScanKeyInfo *keys);
 
 /*
  * RelationBuildPartitionDesc
@@ -1642,7 +1592,7 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
  * the responsibility of the caller to process the argument clauses of each of
  * the OR clauses, which would involve recursively calling this function.
  */
-static int
+int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
 								 List **or_clauses)
@@ -2406,7 +2356,7 @@ partition_cmp_args(Oid partopfamily, Oid partopcintype,
  * Outputs:
  *	Partition set satisfying the keys.
  */
-static Bitmapset *
+Bitmapset *
 get_partitions_for_keys(Relation rel, PartScanKeyInfo *keys)
 {
 	PartitionKey	partkey = RelationGetPartitionKey(rel);
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 853c1f6..9656fa4 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1041,7 +1041,7 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	if (plansource->num_custom_plans < 5)
 		return true;
 
-	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans;
+	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans + 90000;
 
 	/*
 	 * Prefer generic plan if it's less expensive than the average custom
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 81c626f..7956f04 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -111,4 +112,10 @@ extern List *get_proposed_default_constraint(List *new_part_constaints);
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation,
 							List *partclauses);
+extern int classify_partition_bounding_keys(Relation relation, List *clauses,
+								 PartScanKeyInfo *keys, bool *constfalse,
+								 List **or_clauses);
+Bitmapset *get_partitions_for_keys(Relation rel,
+						PartScanKeyInfo *keys);
+
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9c67bd1..3e47de7 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -363,6 +363,51 @@ typedef struct PartitionSchemeData
 
 typedef struct PartitionSchemeData *PartitionScheme;
 
+/*
+ * PartScanKeyInfo
+ *		Bounding scan keys to look up a table's partitions obtained from
+ *		mutually-ANDed clauses containing partitioning-compatible operators
+ */
+typedef struct PartScanKeyInfo
+{
+	/*
+	 * Constants constituting the *whole* partition key compared using
+	 * partitioning-compatible equality operator(s).  When n_eqkeys > 0, other
+	 * keys (minkeys and maxkeys) are irrelevant.
+	 */
+	Datum	eqkeys[PARTITION_MAX_KEYS];
+	int		n_eqkeys;
+
+	/*
+	 * Constants that constitute the lower bound on the partition key or a
+	 * prefix thereof.  The last of those constants is compared using > or >=
+	 * operator compatible with partitioning, making this the lower bound in
+	 * a range query.
+	 */
+	Datum	minkeys[PARTITION_MAX_KEYS];
+	int		n_minkeys;
+	bool	min_incl;
+
+	/*
+	 * Constants that constitute the upper bound on the partition key or a
+	 * prefix thereof.  The last of those constants is compared using < or <=
+	 * operator compatible with partitioning, making this the upper bound in
+	 * a range query.
+	 */
+	Datum	maxkeys[PARTITION_MAX_KEYS];
+	int		n_maxkeys;
+	bool	max_incl;
+
+	/*
+	 * Specifies the type of NullTest that was applied to each of the
+	 * partition key columns or -1 if none was applied.  Partitioning handles
+	 * null partition keys specially depending on the partitioning method in
+	 * use, so get_partitions_for_keys can return partitions according to
+	 * the nullness condition for partition keys.
+	 */
+	NullTestType	keynullness[PARTITION_MAX_KEYS];
+} PartScanKeyInfo;
+
 /*----------
  * RelOptInfo
  *		Per-relation information for planning/optimization
-- 
1.8.3.1

#8Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: Beena Emerson (#7)
Re: [HACKERS] Runtime Partition Pruning

On Tue, Nov 14, 2017 at 11:46 AM, Beena Emerson <memissemerson@gmail.com>
wrote:

PFA the updated patches.

Hi,

I have started testing this along with fast pruning. It is crashing for sql
with subqueries.
one to test case is given below.

CREATE TABLE prun_test_part (empno int, sal int, deptno int) PARTITION BY
RANGE(sal);
CREATE TABLE prun_test_part_p1 PARTITION OF prun_test_part FOR VALUES FROM
(0) TO (100);
CREATE TABLE prun_test_part_p2 PARTITION OF prun_test_part FOR VALUES FROM
(100) TO (200);
CREATE TABLE prun_test_part_p3 PARTITION OF prun_test_part FOR VALUES FROM
(200) TO (300);
CREATE TABLE prun_test_part_p4 PARTITION OF prun_test_part FOR VALUES FROM
(300) TO (400);

INSERT INTO prun_test_part VALUES (10,90,10);
INSERT INTO prun_test_part VALUES (11,100,10);
INSERT INTO prun_test_part VALUES (20,110,20);
INSERT INTO prun_test_part VALUES (21,200,20);
INSERT INTO prun_test_part VALUES (30,210,30);
INSERT INTO prun_test_part VALUES (31,300,30);
INSERT INTO prun_test_part VALUES (50,310,20);

explain (costs off)
SELECT * FROM prun_test_part WHERE sal < (SELECT sal FROM prun_test_part
WHERE sal = 50);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

#9David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#7)
Re: [HACKERS] Runtime Partition Pruning

On 14 November 2017 at 19:16, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patches.

Hi Beena,

Thanks for working on this. I've had a look at the patch to try to
understand how it is working. I found it a bit surprising that the
code assumes it can rely on the order of Append->appendplans matching
what's needed based on the return value of get_partitions_for_keys().

I tried using the following to break this:

drop table if exists ab;
create table ab (a int not null, b int not null) partition by list(a);
create table ab_a2 partition of ab for values in(2) partition by list (b);
create table ab_a2_b1 partition of ab_a2 for values in (1);
create table ab_a2_b2 partition of ab_a2 for values in (2);
create table ab_a2_b3 partition of ab_a2 for values in (3);

create table ab_a1 partition of ab for values in(1) partition by list (b);
create table ab_a1_b1 partition of ab_a1 for values in (1);
create table ab_a1_b2 partition of ab_a1 for values in (2);
create table ab_a1_b3 partition of ab_a1 for values in (3);
create table ab_a3 partition of ab for values in(3) partition by list (b);
create table ab_a3_b1 partition of ab_a3 for values in (1);
create table ab_a3_b2 partition of ab_a3 for values in (2);
create table ab_a3_b3 partition of ab_a3 for values in (3);

prepare ab_q1 (int, int) as select * from ab where a = $1 and b = $2;

explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);

postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8)
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8)
Filter: ((a = 2) AND (b = 3))
(3 rows)

postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..395.10 rows=1 width=8)
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8)
<--------- wrong partition
Filter: ((a = $1) AND (b = $2))
(3 rows)

As soon as we hit the generic plan the wrong partition is selected

I think to do this you're going to have to store some sort of array
that maps the partition index to the subpath in the Append node so you
can correctly identify the subpath based on what you're getting back
from get_partitions_for_keys(). Perhaps what you had worked previously
when we were not returning a Bitmapset with that function.

Once you've got that design worked out I can take another look at this.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#10David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#9)
Re: [HACKERS] Runtime Partition Pruning

On 15 November 2017 at 01:57, David Rowley <david.rowley@2ndquadrant.com> wrote:

I think to do this you're going to have to store some sort of array
that maps the partition index to the subpath in the Append node so you
can correctly identify the subpath based on what you're getting back
from get_partitions_for_keys(). Perhaps what you had worked previously
when we were not returning a Bitmapset with that function.

Once you've got that design worked out I can take another look at this.

I think this is a bit more broken than I originally mentioned above.
The code you have at the moment assumes there will be just a single
partitioned table in the hierarchy. Remember that complex partitioned
hierarchies will be flattened during set_append_rel_pathlist(), so
there may be multiple partitioned relations to search for.

A more simple way to break the patch is to have some constants in the
query to eliminate some of the partitions during planning, leaving
just a few to be eliminated during execution.

Something like:

deallocate ab_q1;
drop table if exists ab;
create table ab (a int not null, b int not null) partition by list(a);
create table ab_a1 partition of ab for values in (1);
create table ab_a2 partition of ab for values in (2);
create table ab_a3 partition of ab for values in (3);
create table ab_a4 partition of ab for values in (4);
create table ab_a5 partition of ab for values in (5);
create table ab_a6 partition of ab for values in (6);
create table ab_a7 partition of ab for values in (7);
create table ab_a8 partition of ab for values in (8);
create table ab_a9 partition of ab for values in (9);
create table ab_a10 partition of ab for values in (10);

prepare ab_q1 (int) as select * from ab where a between 4 and 5 and a = $1;

explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);

explain execute ab_q1 (4); -- TRAP: FailedAssertion("!(n <
list->length)", File: "src/backend/nodes/list.c", Line: 392)

So some sort of hierarchical structure of the partition hierarchy
would need to be stored in the Append node and then you'd need to
search at each level, and then somehow match the results up to the
subpaths that you have in the Append. Although, I'm still not sure
this is the best way to go about this.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#11Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#10)
Re: [HACKERS] Runtime Partition Pruning

On 2017/11/15 8:13, David Rowley wrote:

So some sort of hierarchical structure of the partition hierarchy
would need to be stored in the Append node and then you'd need to
search at each level, and then somehow match the results up to the
subpaths that you have in the Append. Although, I'm still not sure
this is the best way to go about this.

I think we should try to use PartitionDispatch stuff for this somehow,
just like get_partition_for_tuple() does, although I haven't thought very
hard whether that infrastructure would be useful as is.

Thanks,
Amit

#12Beena Emerson
memissemerson@gmail.com
In reply to: Rajkumar Raghuwanshi (#8)
Re: [HACKERS] Runtime Partition Pruning

Hello Rajkumar,

On Tue, Nov 14, 2017 at 2:22 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Nov 14, 2017 at 11:46 AM, Beena Emerson <memissemerson@gmail.com>
wrote:

PFA the updated patches.

Hi,

I have started testing this along with fast pruning. It is crashing for sql
with subqueries.
one to test case is given below.

CREATE TABLE prun_test_part (empno int, sal int, deptno int) PARTITION BY
RANGE(sal);
CREATE TABLE prun_test_part_p1 PARTITION OF prun_test_part FOR VALUES FROM
(0) TO (100);
CREATE TABLE prun_test_part_p2 PARTITION OF prun_test_part FOR VALUES FROM
(100) TO (200);
CREATE TABLE prun_test_part_p3 PARTITION OF prun_test_part FOR VALUES FROM
(200) TO (300);
CREATE TABLE prun_test_part_p4 PARTITION OF prun_test_part FOR VALUES FROM
(300) TO (400);

INSERT INTO prun_test_part VALUES (10,90,10);
INSERT INTO prun_test_part VALUES (11,100,10);
INSERT INTO prun_test_part VALUES (20,110,20);
INSERT INTO prun_test_part VALUES (21,200,20);
INSERT INTO prun_test_part VALUES (30,210,30);
INSERT INTO prun_test_part VALUES (31,300,30);
INSERT INTO prun_test_part VALUES (50,310,20);

explain (costs off)
SELECT * FROM prun_test_part WHERE sal < (SELECT sal FROM prun_test_part
WHERE sal = 50);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

Thanks for your review. I am looking into it.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#13Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#10)
Re: [HACKERS] Runtime Partition Pruning

Hello David,

Thank you for reviewing.

On Wed, Nov 15, 2017 at 4:43 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 15 November 2017 at 01:57, David Rowley <david.rowley@2ndquadrant.com> wrote:

I think to do this you're going to have to store some sort of array
that maps the partition index to the subpath in the Append node so you
can correctly identify the subpath based on what you're getting back
from get_partitions_for_keys(). Perhaps what you had worked previously
when we were not returning a Bitmapset with that function.

Once you've got that design worked out I can take another look at this.

I think this is a bit more broken than I originally mentioned above.
The code you have at the moment assumes there will be just a single
partitioned table in the hierarchy. Remember that complex partitioned
hierarchies will be flattened during set_append_rel_pathlist(), so
there may be multiple partitioned relations to search for.

A more simple way to break the patch is to have some constants in the
query to eliminate some of the partitions during planning, leaving
just a few to be eliminated during execution.

Something like:

deallocate ab_q1;
drop table if exists ab;
create table ab (a int not null, b int not null) partition by list(a);
create table ab_a1 partition of ab for values in (1);
create table ab_a2 partition of ab for values in (2);
create table ab_a3 partition of ab for values in (3);
create table ab_a4 partition of ab for values in (4);
create table ab_a5 partition of ab for values in (5);
create table ab_a6 partition of ab for values in (6);
create table ab_a7 partition of ab for values in (7);
create table ab_a8 partition of ab for values in (8);
create table ab_a9 partition of ab for values in (9);
create table ab_a10 partition of ab for values in (10);

prepare ab_q1 (int) as select * from ab where a between 4 and 5 and a = $1;

explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);

explain execute ab_q1 (4); -- TRAP: FailedAssertion("!(n <
list->length)", File: "src/backend/nodes/list.c", Line: 392)

So some sort of hierarchical structure of the partition hierarchy
would need to be stored in the Append node and then you'd need to
search at each level, and then somehow match the results up to the
subpaths that you have in the Append. Although, I'm still not sure
this is the best way to go about this.

Thank you for your suggestion. I am looking into this and will post a
patch soon.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#14Dilip Kumar
dilipbalaut@gmail.com
In reply to: David Rowley (#10)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Nov 15, 2017 at 4:43 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 15 November 2017 at 01:57, David Rowley <david.rowley@2ndquadrant.com> wrote:

I think to do this you're going to have to store some sort of array
that maps the partition index to the subpath in the Append node so you
can correctly identify the subpath based on what you're getting back
from get_partitions_for_keys(). Perhaps what you had worked previously
when we were not returning a Bitmapset with that function.

Once you've got that design worked out I can take another look at this.

So some sort of hierarchical structure of the partition hierarchy
would need to be stored in the Append node and then you'd need to
search at each level, and then somehow match the results up to the
subpaths that you have in the Append. Although, I'm still not sure
this is the best way to go about this.

Instead of hierarchical structure can’t we maintain an array (one
entry per partition), and whenever any leaf partition’s subpath
is added to the append rel (in function set_append_rel_pathlist) we
can set that subpath number in corresponding array index.

And, later we can add some wrapper over get_partitions_for_key such
that it can recursively traverse the non-leaf partitions (something
like get_partition_for_tuple does as Amit mentioned.). And, ultimately
gives the output as a list of leaf partition's indexes. Then we can
find the sub-plan number by looking into the array.

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

#15Michael Paquier
michael.paquier@gmail.com
In reply to: Beena Emerson (#13)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Nov 15, 2017 at 3:53 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Thank you for your suggestion. I am looking into this and will post a
patch soon.

It has been two weeks since this update and no new patch has showed
up. I am marking the patch as returned with feedback. If you can
produce a new version, of course feel free to post it and register a
new entry in the CF app.
--
Michael

#16Beena Emerson
memissemerson@gmail.com
In reply to: Michael Paquier (#15)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hello,

PFA the new version of the patch which can be applied over v11 patches
of Amit Langote [1]/messages/by-id/62d21a7b-fea9-f2d7-c33a-8caa12eca612@lab.ntt.co.jp. The patch has been completely modified and the
0001 patch of previous series is no longer required. As mentioned
above, I have used the PartitionDispatchInfo and an array to which
holds the actual subplan index.

This one addresses the problems mentioned above but still needs to be
thoroughly tested. I will soon post a new patch rebased over Amit's
v13 patches[2]/messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp soon with additional code comments.

[1]: /messages/by-id/62d21a7b-fea9-f2d7-c33a-8caa12eca612@lab.ntt.co.jp
[2]: /messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning_v3.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning_v3.patchDownload
From c40fce95dfb66faf4aae319f1a424dd4eb39028a Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Wed, 29 Nov 2017 16:58:47 +0530
Subject: [PATCH] Implement Runtime Partition Pruning

Patch by: Beena Emerson, Dilip Kumar
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c         | 111 ++++++++++++--
 src/backend/executor/nodeAppend.c       | 263 ++++++++++++++++++++++++++++++--
 src/backend/nodes/copyfuncs.c           |   5 +
 src/backend/optimizer/path/allpaths.c   | 150 +++++++++++++++---
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  24 +++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  14 ++
 src/backend/optimizer/util/pathnode.c   |  30 +++-
 src/backend/optimizer/util/relnode.c    |  23 ++-
 src/backend/utils/cache/plancache.c     |   2 +-
 src/include/catalog/partition.h         |   6 +-
 src/include/nodes/execnodes.h           |   6 +
 src/include/nodes/plannodes.h           |   6 +
 src/include/nodes/relation.h            |  12 ++
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   4 +-
 18 files changed, 603 insertions(+), 62 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index c58c735..5a07a32 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -33,6 +33,7 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -241,19 +242,22 @@ static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
 
 static Bitmapset *get_partitions_from_clauses_guts(Relation relation,
-								List *clauses);
+								 List *clauses, ParamListInfo prmList,
+								 ExprContext *econtext);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses,
+								 ParamListInfo prmList, ExprContext *econtext);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
 static bool partition_cmp_args(Oid partopfamily, Oid partopcintype,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
 				   bool *result);
-static bool partkey_datum_from_expr(const Expr *expr, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
+static bool partkey_datum_from_expr(const Expr *expr, Datum *value,
+						ParamListInfo prmList, ExprContext *econtext);
 
 /*
  * RelationBuildPartitionDesc
@@ -1536,6 +1540,34 @@ get_partition_dispatch_recurse(Relation rel, Relation parent,
 	}
 }
 
+void
+get_leaf_part_recurse(Relation rel, List **leaf_part_oids)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+	int			i;
+
+	check_stack_depth();
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		Oid			partrelid = partdesc->oids[i];
+
+		if (get_rel_relkind(partrelid) != RELKIND_PARTITIONED_TABLE)
+			*leaf_part_oids = lappend_oid(*leaf_part_oids, partrelid);
+		else
+		{
+			/*
+			 * We assume all tables in the partition tree were already locked
+			 * by the caller.
+			 */
+			Relation	partrel = heap_open(partrelid, NoLock);
+
+			get_leaf_part_recurse(partrel, leaf_part_oids);
+			heap_close(partrel, NoLock);
+		}
+	}
+}
+
 /*
  * get_partitions_from_clauses
  *		Determine the set of partitions of relation that will satisfy all
@@ -1545,7 +1577,8 @@ get_partition_dispatch_recurse(Relation rel, Relation parent,
  *	A Bitmapset containing indexes of all selected partitions.
  */
 Bitmapset *
-get_partitions_from_clauses(Relation relation, List *partclauses)
+get_partitions_from_clauses(Relation relation, List *partclauses,
+							 ParamListInfo prmList, ExprContext *econtext)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1554,7 +1587,7 @@ get_partitions_from_clauses(Relation relation, List *partclauses)
 		partconstr = (List *) expression_planner((Expr *) partconstr);
 	partclauses = list_concat(partclauses, partconstr);
 
-	result = get_partitions_from_clauses_guts(relation, partclauses);
+	result = get_partitions_from_clauses_guts(relation, partclauses, prmList, econtext);
 
 	return result;
 }
@@ -1569,7 +1602,8 @@ get_partitions_from_clauses(Relation relation, List *partclauses)
  * Return value is a Bitmapset containing the indexes of selected partitions.
  */
 static Bitmapset *
-get_partitions_from_clauses_guts(Relation relation, List *clauses)
+get_partitions_from_clauses_guts(Relation relation, List *clauses,
+								 ParamListInfo prmList, ExprContext *econtext)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
 	Bitmapset *result = NULL;
@@ -1581,7 +1615,8 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 
 	nkeys = classify_partition_bounding_keys(relation, clauses,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, prmList,
+											 econtext);
 	/*
 	 * Only look up in the partition decriptor if the query provides
 	 * constraints on the keys at all.
@@ -1604,7 +1639,8 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 			Bitmapset *arg_partset;
 
 			arg_partset = get_partitions_from_clauses_guts(relation,
-														   list_make1(orarg));
+														   list_make1(orarg),
+														   prmList, econtext);
 
 			/* Combine partition sets obtained from mutually ORed clauses. */
 			or_partset = bms_union(or_partset, arg_partset);
@@ -1645,7 +1681,8 @@ get_partitions_from_clauses_guts(Relation relation, List *clauses)
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses,
+								 ParamListInfo prmList, ExprContext *econtext)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -1694,6 +1731,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				continue;
 			}
 		}
+		else if (IsA(lfirst(lc), ExprState))
+			clause = ((ExprState *) lfirst(lc))->expr;
 		else
 			clause = (Expr *) lfirst(lc);
 
@@ -2138,7 +2177,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_eqkeys; i++)
 		{
-			if (partkey_datum_from_expr(eqkey_exprs[i], &value))
+			if (partkey_datum_from_expr(eqkey_exprs[i], &value,
+										prmList, econtext))
 			{
 				keys->eqkeys[i] = value;
 				n_datums_resolved++;
@@ -2150,7 +2190,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_minkeys; i++)
 		{
-			if (partkey_datum_from_expr(minkey_exprs[i], &value))
+			if (partkey_datum_from_expr(minkey_exprs[i], &value,
+										prmList, econtext))
 			{
 				keys->minkeys[i] = value;
 				n_datums_resolved++;
@@ -2163,7 +2204,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 		n_datums_resolved = 0;
 		for (i = 0; i < n_maxkeys; i++)
 		{
-			if (partkey_datum_from_expr(maxkey_exprs[i], &value))
+			if (partkey_datum_from_expr(maxkey_exprs[i], &value,
+										prmList, econtext))
 			{
 				keys->maxkeys[i] = value;
 				n_datums_resolved++;
@@ -2186,7 +2228,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
  *		Extract constant value from expr and set *datum to that value
  */
 static bool
-partkey_datum_from_expr(const Expr *expr, Datum *value)
+partkey_datum_from_expr(const Expr *expr, Datum *value, ParamListInfo prmList,
+						ExprContext *econtext)
 {
 	/*
 	 * Add more expression types here as needed to support higher-level
@@ -2198,6 +2241,42 @@ partkey_datum_from_expr(const Expr *expr, Datum *value)
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmList)
+					{
+						Node	   *n = eval_const_expressions_from_list(prmList, (Node *) expr);
+
+						if (IsA(n, Const))
+						{
+							*value = ((Const *) n)->constvalue;
+							return true;
+						}
+					}
+					return false;
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
+			return false;
+
 		default:
 			return false;
 	}
@@ -2353,9 +2432,11 @@ partition_cmp_args(Oid partopfamily, Oid partopcintype,
 			rightarg_const;
 
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
-	if (!partkey_datum_from_expr(leftarg->constarg, &leftarg_const))
+	if (!partkey_datum_from_expr(leftarg->constarg, &leftarg_const,
+								 NULL, NULL))
 		return false;
-	if (!partkey_datum_from_expr(rightarg->constarg, &rightarg_const))
+	if (!partkey_datum_from_expr(rightarg->constarg, &rightarg_const,
+								 NULL, NULL))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..2890b3a 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,12 @@
 
 #include "postgres.h"
 
+#include "catalog/pg_inherits_fn.h"
+#include "nodes/relation.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "optimizer/clauses.h"
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool exec_append_initialize_next(AppendState *appendstate);
@@ -107,6 +110,24 @@ exec_append_initialize_next(AppendState *appendstate)
 	}
 }
 
+static List *
+initClauses(PlanState *parent, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		new_list = lappend(new_list, ExecInitExpr(val, parent));
+	}
+	return new_list;
+}
+
 /* ----------------------------------------------------------------
  *		ExecInitAppend
  *
@@ -165,25 +186,134 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
-	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
-	 */
-	i = 0;
-	foreach(lc, node->appendplans)
+	if (node->extern_quals)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		List	   *temp_appendplans = NIL;
+		Append	   *temp_node;
+		Relation	rel;
+		Bitmapset  *partset;
+		Bitmapset  *subplan_indexes;
+		int			i,
+					cur_index;
+		PartitionDispatch *pd,
+				   *p1,
+					parent;
+		int			num_parted;
+		List	   *leaf_part_oids = NIL;
+		List	   *parents = NIL;
+
+		rel = relation_open(node->parentoid, NoLock);
+
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		(void) find_all_inheritors(RelationGetRelid(rel), AccessShareLock, NULL);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		parents = lappend(parents, &pd[0]);
+		do
+		{
+			p1 = linitial(parents);
+			parent = *p1;
+
+			partset = get_partitions_from_clauses(parent->reldesc,
+												  list_copy(node->extern_quals),
+												  estate->es_param_list_info,
+												  NULL);
+
+			if (!bms_is_empty(partset))
+			{
+				i = 0;
+				while ((cur_index = bms_first_member(partset)) >= 0)
+				{
+					if (cur_index < 0)
+					{
+					//	PartitionDesc *partdesc = parent->partdesc;
+
+					//	subplan_indexes = bms_make_singleton(partdesc->boundinfo->default_index);
+					}
+
+					else if (parent->indexes[cur_index] >= 0)
+					{
+						subplan_indexes = bms_make_singleton(parent->indexes[cur_index]);
+						break;
+					}
+					else
+						parents = lappend(parents, &pd[-parent->indexes[cur_index]]);
+				}
+			}
+
+			parents = list_delete_first(parents);
+		} while (parents);
+
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+
+		if (!bms_is_empty(subplan_indexes))
+		{
+			int			j;
+			int			i = 0;
+
+			while ((j = bms_first_member(subplan_indexes)) >= 0)
+			{
+				int			index = node->append_paths_array[j];
+
+				if (index >= 0)
+				{
+					Plan	   *initNode = (Plan *) list_nth(node->appendplans,
+															 index);
+
+					temp_appendplans = lappend(temp_appendplans, initNode);
+					appendplanstates[i++] = ExecInitNode(initNode, estate,
+														 eflags);
+				}
+			}
+		}
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
-		i++;
+		relation_close(rel, NoLock);
+		/* create new AppendState for our append node */
+		temp_node = copyObject(node);
+		temp_node->appendplans = temp_appendplans;
+		((Plan *) temp_node)->plan_rows = i;
+		appendstate->as_nplans = i;
+		appendstate->ps.plan = (Plan *) temp_node;
+		appendstate->ps.state = estate;
+		appendstate->ps.ExecProcNode = ExecAppend;
+		appendstate->appendplans = appendplanstates;
 	}
+	else
+	{
+		appendstate->parentoid = node->parentoid;
+		ExecAssignExprContext(estate, &appendstate->ps);
 
-	/*
-	 * initialize output tuple type
-	 */
+		/*
+		 * call ExecInitNode on each of the plans to be executed and save the
+		 * results into the array "appendplans".
+		 */
+		i = 0;
+		foreach(lc, node->appendplans)
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+
+			appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			i++;
+		}
+	}
+
+	if (node->join_clauses)
+		appendstate->join_clauses = initClauses((PlanState *) appendstate,
+											 node->join_clauses);
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
+	appendstate->array_size = node->array_size;
+	appendstate->append_paths_array = node->append_paths_array;
+
 	/*
 	 * initialize to scan first subplan
 	 */
@@ -237,7 +367,22 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->index)
+			{
+				if (++node->as_whichpartition < list_length(node->index))
+					node->as_whichplan = list_nth_int(node->index,
+													  node->as_whichpartition);
+
+				return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -280,6 +425,89 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	if (node->ps.chgParam != NULL && node->join_clauses)
+	{
+		Bitmapset  *partset,
+				   *subplan_indexes = NULL;
+		Relation	rel;
+		PartitionDispatch *pd,
+					parent,
+				   *p1;
+		List	   *parents = NIL;
+		List	   *leaf_part_oids = NIL;
+		int			cur_index,
+					num_parted;
+
+		rel = relation_open(node->parentoid, NoLock);
+
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		(void) find_all_inheritors(RelationGetRelid(rel), AccessShareLock, NULL);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		relation_close(rel, NoLock);
+		parents = lappend(parents, &pd[0]);
+		node->index = NIL;
+		do
+		{
+			p1 = linitial(parents);
+			parent = *p1;
+
+			partset = get_partitions_from_clauses(parent->reldesc,
+												  list_copy(node->join_clauses),
+												  NULL,
+												  node->ps.ps_ExprContext);
+
+			if (!bms_is_empty(partset))
+			{
+				i = 0;
+				while ((cur_index = bms_first_member(partset)) >= 0)
+				{
+					if (cur_index < 0)
+					{
+						//PartitionDesc *partdesc = parent->partdesc;
+
+					//	subplan_indexes = bms_make_singleton(partdesc->boundinfo->default_index);
+					}
+
+					else if (parent->indexes[cur_index] >= 0)
+					{
+						subplan_indexes =
+							bms_make_singleton(parent->indexes[cur_index]);
+						break;
+					}
+					else
+						parents =
+							lappend(parents, &pd[-parent->indexes[cur_index]]);
+				}
+			}
+
+			parents = list_delete_first(parents);
+		} while (parents);
+
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+
+		if (!bms_is_empty(subplan_indexes))
+		{
+			int			j;
+
+			while ((j = bms_first_member(subplan_indexes)) >= 0)
+			{
+				int			index = node->append_paths_array[j];
+
+				if (index >= 0)
+					node->index = lappend_int(node->index, index);
+			}
+		}
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +526,13 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->index)
+	{
+		node->as_whichplan = linitial_int(node->index);
+		node->as_whichpartition = 0;
+	}
+	else
+		node->as_whichplan = 0;
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index c1a83ca..63b206f 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,11 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(extern_quals);
+	COPY_NODE_FIELD(join_clauses);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_POINTER_FIELD(append_paths_array, from->array_size * sizeof(int));
+	COPY_SCALAR_FIELD(array_size);
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a3d0468..b7db443 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -18,10 +18,12 @@
 #include <limits.h>
 #include <math.h>
 
+#include "catalog/partition.h"
 #include "access/sysattr.h"
 #include "access/tsmapi.h"
 #include "catalog/partition.h"
 #include "catalog/pg_class.h"
+#include "catalog/pg_inherits_fn.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_proc.h"
@@ -140,11 +142,13 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 						List *live_childrels);
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte);
+						  RangeTblEntry *rte,  List *chk_clauses);
 static List *match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse);
+static int	list_member_oid_index(List *list, Oid datum);
 
 
 /*
@@ -288,6 +292,22 @@ set_base_rel_sizes(PlannerInfo *root)
 		if (root->glob->parallelModeOK)
 			set_rel_consider_parallel(root, rel, rte);
 
+		if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			List	   *leaf_parts = NIL;
+			Relation	parent = relation_open(rte->relid, NoLock);
+			int			i;
+
+			(void) find_all_inheritors(RelationGetRelid(parent), AccessShareLock, NULL);
+			get_leaf_part_recurse(parent, &leaf_parts);
+			root->array_size = list_length(leaf_parts);
+			root->leaf_node_oids = leaf_parts;
+			root->append_paths_array = palloc0(root->array_size * sizeof(int));
+			for (i = 0; i < root->array_size; i++)
+				root->append_paths_array[i] = -1;
+			root->append_count = 0;
+			relation_close(parent, NoLock);
+		}
 		set_rel_size(root, rel, rti, rte);
 	}
 }
@@ -349,6 +369,41 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	{
 		/* It's an "append relation", process accordingly */
 		set_append_rel_size(root, rel, rti, rte);
+
+		if (rel->joininfo && rel->part_scheme)
+		{
+			List	   *partclauses;
+			bool		contains_param,
+						contains_const,
+						constfalse;
+
+			/*
+			 * Get the clauses that match the partition key, including
+			 * information about any nullness tests against partition keys.
+			 * Set keynullness to a invalid value of NullTestType, which 0 is
+			 * not.
+			 */
+			partclauses = match_clauses_to_partkey(rel,
+												   list_copy(rel->joininfo),
+												   &contains_param,
+												   &contains_const,
+												   &constfalse);
+
+			if (partclauses != NIL)
+			{
+				ListCell   *lc;
+
+				foreach(lc, partclauses)
+				{
+					Node	   *n = lfirst(lc);
+
+					if (!list_member(root->join_append_clauses, n));
+					root->join_append_clauses = lappend(root->join_append_clauses, n);
+				}
+			}
+
+		}
+
 	}
 	else
 	{
@@ -855,6 +910,7 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 	rel->fdwroutine->GetForeignPaths(root, rel, rte->relid);
 }
 
+
 /*
  * get_append_rel_partitions
  *		Return the list of partitions of rel that pass the clauses mentioned
@@ -865,24 +921,26 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 static List *
 get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte)
+						  RangeTblEntry *rte,  List *chk_clauses)
 {
 	Relation	parent = heap_open(rte->relid, NoLock);
 	PartitionDesc partdesc = RelationGetPartitionDesc(parent);
-	List   *partclauses;
-	List   *result = NIL;
-	int		i;
-	Bitmapset *partindexes = NULL;
-	bool	contains_const,
-			constfalse;
+	List	   *partclauses;
+	List	   *result = NIL;
+	int			i;
+	Bitmapset  *partindexes = NULL;
+	bool		contains_param,
+				contains_const,
+				constfalse;
 
 	/*
 	 * Get the clauses that match the partition key, including information
-	 * about any nullness tests against partition keys.  Set keynullness to
-	 * a invalid value of NullTestType, which 0 is not.
+	 * about any nullness tests against partition keys.  Set keynullness to a
+	 * invalid value of NullTestType, which 0 is not.
 	 */
 	partclauses = match_clauses_to_partkey(rel,
 										   list_copy(rel->baserestrictinfo),
+										   &contains_param,
 										   &contains_const,
 										   &constfalse);
 
@@ -891,11 +949,26 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * the same to prune partitions right away.
 	 */
 	if (partclauses != NIL && contains_const && !constfalse)
-		partindexes = get_partitions_from_clauses(parent, partclauses);
-	else if (!constfalse)
+		partindexes = get_partitions_from_clauses(parent, partclauses, NULL, NULL);
+	else if (!constfalse && !contains_param)
 		/* No clauses to prune paritions, so scan all partitions. */
 		partindexes = bms_add_range(partindexes, 0, partdesc->nparts - 1);
 
+	if (partclauses != NIL && contains_param)
+	{
+		ListCell   *lc;
+
+		foreach(lc, partclauses)
+		{
+			Node	   *n = lfirst(lc);
+
+			if (!list_member(root->extern_clauses, n));
+			root->extern_clauses = lappend(root->extern_clauses, n);
+		}
+		if (!partindexes)
+			partindexes = bms_add_range(partindexes, 0, partdesc->nparts - 1);
+	}
+
 	/* Fetch the partition appinfos. */
 	while ((i = bms_first_member(partindexes)) >= 0)
 	{
@@ -951,10 +1024,13 @@ get_append_rel_partitions(PlannerInfo *root,
  *
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
+ *
+ * If the list has a param, *contains_param is set
  */
 static List *
 match_clauses_to_partkey(RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -962,6 +1038,7 @@ match_clauses_to_partkey(RelOptInfo *rel,
 	List	   *result = NIL;
 	ListCell   *lc;
 
+	*contains_param = false;
 	*contains_const = false;
 	*constfalse = false;
 
@@ -1002,6 +1079,7 @@ match_clauses_to_partkey(RelOptInfo *rel,
 			if (or_clause((Node *) clause) &&
 				match_clauses_to_partkey(rel,
 									 list_copy(((BoolExpr *) clause)->args),
+										 contains_param,
 										 &contains_const1,
 										 &constfalse1) != NIL)
 			{
@@ -1079,6 +1157,9 @@ match_clauses_to_partkey(RelOptInfo *rel,
 					/* Neither argument matches the partition key. */
 					continue;
 
+				if (IsA(constexpr, Param))
+					*contains_param = true;
+
 				/*
 				 * Only allow strict operators to think sanely about the
 				 * behavior with null arguments.
@@ -1230,6 +1311,22 @@ match_clauses_to_partkey(RelOptInfo *rel,
 	return result;
 }
 
+static int
+list_member_oid_index(List *list, Oid datum)
+{
+	int			i = 0;
+	const ListCell *cell;
+
+	foreach(cell, list)
+	{
+		if (lfirst_oid(cell) == datum)
+			return i;
+		i++;
+	}
+
+	return -1;
+}
+
 /*
  * set_append_rel_size
  *	  Set size estimates for a simple "append relation"
@@ -1245,7 +1342,8 @@ static void
 set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					Index rti, RangeTblEntry *rte)
 {
-	List	   *rel_appinfos = NIL;
+	List	   *rel_appinfos = NIL,
+			   *chk_clauses = NIL;
 	int			parentRTindex = rti;
 	bool		has_live_children;
 	double		parent_rows;
@@ -1261,9 +1359,9 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 
 	if (rte->relkind != RELKIND_PARTITIONED_TABLE)
 	{
-		foreach (l, root->append_rel_list)
+		foreach(l, root->append_rel_list)
 		{
-			AppendRelInfo   *appinfo = lfirst(l);
+			AppendRelInfo *appinfo = lfirst(l);
 
 			/* append_rel_list contains all append rels; ignore others */
 			if (appinfo->parent_relid == parentRTindex)
@@ -1272,7 +1370,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	}
 	else
 	{
-		rel_appinfos = get_append_rel_partitions(root, rel, rte);
+		rel_appinfos = get_append_rel_partitions(root, rel, rte, chk_clauses);
 		rel->live_partitioned_rels = list_make1_int(rti);
 	}
 
@@ -1495,8 +1593,8 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 		if (childrel->part_scheme && rel->part_scheme)
 		{
 			rel->live_partitioned_rels =
-					list_concat(rel->live_partitioned_rels,
-								list_copy(childrel->live_partitioned_rels));
+				list_concat(rel->live_partitioned_rels,
+							list_copy(childrel->live_partitioned_rels));
 		}
 
 		/*
@@ -1623,6 +1721,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		int			childRTindex;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
+		int			index;
 
 		/* append_rel_list contains all append rels; ignore others */
 		if (appinfo->parent_relid != parentRTindex)
@@ -1653,6 +1752,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* Only consider non dummy children */
+		index = list_member_oid_index(root->leaf_node_oids, childRTE->relid);
+		if (index >= 0)
+			root->append_paths_array[index] = root->append_count++;
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
@@ -1714,7 +1818,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		rte = planner_rt_fetch(rel->relid, root);
 		if (rte->rtekind == RTE_RELATION &&
 			rte->relkind == RELKIND_PARTITIONED_TABLE)
-		partitioned_rels = rel->live_partitioned_rels;
+			partitioned_rels = rel->live_partitioned_rels;
 	}
 	else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
 	{
@@ -1834,7 +1938,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1861,7 +1965,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1915,7 +2019,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2152,7 +2256,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 7356683..af7e790 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9c74e39..1670998 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,24 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static List *
+replace_partition_nestloop_params(PlannerInfo *root, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Node	   *n = lfirst(lc);
+
+		new_list = lappend(new_list, replace_nestloop_params(root, n));
+	}
+	return new_list;
+}
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1081,12 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	plan->extern_quals = best_path->extern_quals;
+	plan->join_clauses = replace_partition_nestloop_params(root, best_path->join_clauses);
+	plan->parentoid = best_path->parentoid;
+	plan->array_size = best_path->array_size;
+	plan->append_paths_array = best_path->append_paths_array;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index fd0e483..8a8876f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3675,7 +3675,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 30cdd3d..2f46932 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -5197,3 +5197,17 @@ tlist_matches_coltypelist(List *tlist, List *coltypelist)
 
 	return true;
 }
+
+Node *
+eval_const_expressions_from_list(ParamListInfo prmList, Node *node)
+{
+	eval_const_expressions_context context;
+
+	if (prmList)
+		context.boundParams = prmList;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 36ec025..889419d 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,34 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	if (root)
+	{
+		pathnode->append_paths_array = root->append_paths_array;
+		pathnode->array_size = root->array_size;
+
+	}
+	if (root && (required_outer || root->extern_clauses))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				if (root->join_append_clauses)
+					pathnode->join_clauses = root->join_append_clauses;
+				/* Set et_keys for extern params */
+				if (root->extern_clauses)
+					pathnode->extern_quals = root->extern_clauses;
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index b06696b..9abc79c 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1556,6 +1556,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1567,11 +1570,29 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+
+	/*
+	 * Generally for appendrel we don't fetch the clause from the join clause
+	 * (only we do so for baserel), but for identifying whether the appendrel
+	 * is applicable for runtime pruning or not.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 81c626f..2bad8f1 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -110,5 +111,8 @@ extern List *get_proposed_default_constraint(List *new_part_constaints);
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation,
-							List *partclauses);
+							List *partclauses,
+							ParamListInfo prmList,
+							ExprContext *econtext);
+extern void		get_leaf_part_recurse(Relation rel, List **leaf_part_oids);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..620fc25 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,12 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Oid			parentoid;
+	List	   *index;			/* subplan indexes to scan for runtime pruning */
+	int			as_whichpartition;	/* current partition scanned from list */
+	List	   *join_clauses;
+	int		   *append_paths_array;
+	int			array_size;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index dd74efa..217a841 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,11 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+	Oid			parentoid;
+	List	   *extern_quals;
+	List	   *join_clauses;
+	int		   *append_paths_array;
+	int			array_size;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9c67bd1..166e964 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -168,6 +168,7 @@ typedef struct PlannerInfo
 	 */
 	List	   *plan_params;	/* list of PlannerParamItems, see below */
 	Bitmapset  *outer_params;
+	List	   *extern_clauses;
 
 	/*
 	 * simple_rel_array holds pointers to "base rels" and "other rels" (see
@@ -317,6 +318,12 @@ typedef struct PlannerInfo
 
 	/* optional private data for join_search_hook, e.g., GEQO */
 	void	   *join_search_private;
+
+	List	   *leaf_node_oids;
+	List	   *join_append_clauses;
+	int			array_size;
+	int			append_count;
+	int		   *append_paths_array;
 } PlannerInfo;
 
 
@@ -1289,6 +1296,11 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+	List	   *extern_quals;
+	List	   *join_clauses;
+	int		   *append_paths_array;
+	int			array_size;
+	Oid			parentoid;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index e367221..c7f5262 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prm_list, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..7c1fbfd 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,8 +63,8 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
+				   List *subpaths, Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
-- 
1.8.3.1

#17Beena Emerson
memissemerson@gmail.com
In reply to: Rajkumar Raghuwanshi (#8)
Re: [HACKERS] Runtime Partition Pruning

Hello Rajkumar,

On Tue, Nov 14, 2017 at 2:22 PM, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

On Tue, Nov 14, 2017 at 11:46 AM, Beena Emerson <memissemerson@gmail.com>
wrote:

PFA the updated patches.

Hi,

I have started testing this along with fast pruning. It is crashing for sql
with subqueries.
one to test case is given below.

CREATE TABLE prun_test_part (empno int, sal int, deptno int) PARTITION BY
RANGE(sal);
CREATE TABLE prun_test_part_p1 PARTITION OF prun_test_part FOR VALUES FROM
(0) TO (100);
CREATE TABLE prun_test_part_p2 PARTITION OF prun_test_part FOR VALUES FROM
(100) TO (200);
CREATE TABLE prun_test_part_p3 PARTITION OF prun_test_part FOR VALUES FROM
(200) TO (300);
CREATE TABLE prun_test_part_p4 PARTITION OF prun_test_part FOR VALUES FROM
(300) TO (400);

INSERT INTO prun_test_part VALUES (10,90,10);
INSERT INTO prun_test_part VALUES (11,100,10);
INSERT INTO prun_test_part VALUES (20,110,20);
INSERT INTO prun_test_part VALUES (21,200,20);
INSERT INTO prun_test_part VALUES (30,210,30);
INSERT INTO prun_test_part VALUES (31,300,30);
INSERT INTO prun_test_part VALUES (50,310,20);

explain (costs off)
SELECT * FROM prun_test_part WHERE sal < (SELECT sal FROM prun_test_part
WHERE sal = 50);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

With the new patch, the output is as follows:

postgres=# explain (costs off)
SELECT * FROM prun_test_part WHERE sal < (SELECT sal FROM
prun_test_part WHERE sal = 50);
QUERY PLAN
-----------------------------------------------------------------
Append
InitPlan 1 (returns $0)
-> Append
-> Seq Scan on prun_test_part_p1 prun_test_part_p1_1
Filter: (sal = 50)
-> Seq Scan on prun_test_part_p1
Filter: (sal < $0)
(7 rows)

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#18Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#9)
Re: [HACKERS] Runtime Partition Pruning

On Tue, Nov 14, 2017 at 6:27 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 14 November 2017 at 19:16, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patches.

Hi Beena,

Thanks for working on this. I've had a look at the patch to try to
understand how it is working. I found it a bit surprising that the
code assumes it can rely on the order of Append->appendplans matching
what's needed based on the return value of get_partitions_for_keys().

I tried using the following to break this:

drop table if exists ab;
create table ab (a int not null, b int not null) partition by list(a);
create table ab_a2 partition of ab for values in(2) partition by list (b);
create table ab_a2_b1 partition of ab_a2 for values in (1);
create table ab_a2_b2 partition of ab_a2 for values in (2);
create table ab_a2_b3 partition of ab_a2 for values in (3);

create table ab_a1 partition of ab for values in(1) partition by list (b);
create table ab_a1_b1 partition of ab_a1 for values in (1);
create table ab_a1_b2 partition of ab_a1 for values in (2);
create table ab_a1_b3 partition of ab_a1 for values in (3);
create table ab_a3 partition of ab for values in(3) partition by list (b);
create table ab_a3_b1 partition of ab_a3 for values in (1);
create table ab_a3_b2 partition of ab_a3 for values in (2);
create table ab_a3_b3 partition of ab_a3 for values in (3);

prepare ab_q1 (int, int) as select * from ab where a = $1 and b = $2;

explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);
explain execute ab_q1 (2,3);

postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8)
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8)
Filter: ((a = 2) AND (b = 3))
(3 rows)

postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..395.10 rows=1 width=8)
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8)
<--------- wrong partition
Filter: ((a = $1) AND (b = $2))
(3 rows)

With the new patch, the output is:
postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8)
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8)
Filter: ((a = 2) AND (b = 3))
(3 rows)

postgres=# explain execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------
Append (cost=0.00..395.10 rows=4 width=8)
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8)
Filter: ((a = $1) AND (b = $2))
(3 rows)

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#19Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#10)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Wed, Nov 15, 2017 at 4:43 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 15 November 2017 at 01:57, David Rowley <david.rowley@2ndquadrant.com> wrote:

I think to do this you're going to have to store some sort of array
that maps the partition index to the subpath in the Append node so you
can correctly identify the subpath based on what you're getting back
from get_partitions_for_keys(). Perhaps what you had worked previously
when we were not returning a Bitmapset with that function.

Once you've got that design worked out I can take another look at this.

I think this is a bit more broken than I originally mentioned above.
The code you have at the moment assumes there will be just a single
partitioned table in the hierarchy. Remember that complex partitioned
hierarchies will be flattened during set_append_rel_pathlist(), so
there may be multiple partitioned relations to search for.

A more simple way to break the patch is to have some constants in the
query to eliminate some of the partitions during planning, leaving
just a few to be eliminated during execution.

Something like:

deallocate ab_q1;
drop table if exists ab;
create table ab (a int not null, b int not null) partition by list(a);
create table ab_a1 partition of ab for values in (1);
create table ab_a2 partition of ab for values in (2);
create table ab_a3 partition of ab for values in (3);
create table ab_a4 partition of ab for values in (4);
create table ab_a5 partition of ab for values in (5);
create table ab_a6 partition of ab for values in (6);
create table ab_a7 partition of ab for values in (7);
create table ab_a8 partition of ab for values in (8);
create table ab_a9 partition of ab for values in (9);
create table ab_a10 partition of ab for values in (10);

prepare ab_q1 (int) as select * from ab where a between 4 and 5 and a = $1;

explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);
explain execute ab_q1 (4);

explain execute ab_q1 (4); -- TRAP: FailedAssertion("!(n <
list->length)", File: "src/backend/nodes/list.c", Line: 392)

This is handled in the new patch.

postgres=# explain execute ab_q1 (4);
QUERY PLAN
------------------------------------------------------------
Append (cost=0.00..49.55 rows=1 width=8)
-> Seq Scan on ab_a4 (cost=0.00..49.55 rows=1 width=8)
Filter: ((a >= 4) AND (a <= 5) AND (a = 4))
(3 rows)

postgres=# explain execute ab_q1 (4);
QUERY PLAN
------------------------------------------------------------
Append (cost=0.00..99.10 rows=1 width=8)
-> Seq Scan on ab_a4 (cost=0.00..49.55 rows=1 width=8)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
(3 rows)

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#20Beena Emerson
memissemerson@gmail.com
In reply to: Michael Paquier (#15)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Wed, Nov 29, 2017 at 7:11 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:

On Wed, Nov 15, 2017 at 3:53 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Thank you for your suggestion. I am looking into this and will post a
patch soon.

It has been two weeks since this update and no new patch has showed
up. I am marking the patch as returned with feedback. If you can
produce a new version, of course feel free to post it and register a
new entry in the CF app.

I have moved it to next CF with status needs review.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#21Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#19)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hello,

PFA the updated patch rebased over Amit's v13 patches [1]/messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp part of
which is committed. This also fixes few bugs I found.
The InitPlans require execPlan which is not set during ExecInitAppend
and so the evaluation of extern quals is moved from ExecInitAppend to
ExecAppend. This changes the output of explain but only the correct
partition(s) are scanned.

David Q1:
postgres=# explain analyse execute ab_q1 (3,3); --const
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8) (actual time=0.006..0.006
rows=0 loops=1)
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.005..0.005 rows=0 loops=1)
Filter: ((a = 3) AND (b = 3))
Planning time: 0.588 ms
Execution time: 0.043 ms
(5 rows)

postgres=# explain analyse execute ab_q1 (3,3); --Param only
ab_a3_b3 plan is executed
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..395.10 rows=9 width=8) (actual time=0.119..0.119
rows=0 loops=1)
-> Seq Scan on ab_a1_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.006..0.006 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.828 ms
Execution time: 0.234 ms
(21 rows)

David Q1
postgres=# explain analyse execute ab_q1 (4); -- Const
QUERY PLAN
------------------------------------------------------------------------------------------------------
Append (cost=0.00..49.55 rows=1 width=8) (actual time=0.005..0.005
rows=0 loops=1)
-> Seq Scan on ab_a4 (cost=0.00..49.55 rows=1 width=8) (actual
time=0.004..0.004 rows=0 loops=1)
Filter: ((a >= 4) AND (a <= 5) AND (a = 4))
Planning time: 0.501 ms
Execution time: 0.039 ms
(5 rows)

postgres=# explain analyse execute ab_q1 (4); --Param
QUERY PLAN
------------------------------------------------------------------------------------------------------
Append (cost=0.00..99.10 rows=2 width=8) (actual time=0.063..0.063
rows=0 loops=1)
-> Seq Scan on ab_a4 (cost=0.00..49.55 rows=1 width=8) (actual
time=0.004..0.004 rows=0 loops=1)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
-> Seq Scan on ab_a5 (cost=0.00..49.55 rows=1 width=8) (never executed)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
Planning time: 0.563 ms
Execution time: 0.111 ms

I am still working on the patch to add more comments and regression
tests but comments on the code is welcome.

[1]: /messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning_v4.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning_v4.patchDownload
From 0a6e1da7fab7b2100ff14d202b8577e0f736f002 Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Fri, 1 Dec 2017 16:35:40 +0530
Subject: [PATCH] Implement Runtime Partition Pruning

Patch by: Beena Emerson, Dilip Kumar
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c         | 116 ++++++++++++++---
 src/backend/executor/execPartition.c    |   4 +-
 src/backend/executor/nodeAppend.c       | 219 +++++++++++++++++++++++++++++++-
 src/backend/nodes/copyfuncs.c           |   5 +
 src/backend/optimizer/path/allpaths.c   | 131 ++++++++++++++++---
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  24 ++++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  14 ++
 src/backend/optimizer/util/pathnode.c   |  30 ++++-
 src/backend/optimizer/util/relnode.c    |  23 +++-
 src/backend/utils/cache/plancache.c     |   2 +-
 src/include/catalog/partition.h         |   4 +-
 src/include/executor/execPartition.h    |   3 +
 src/include/nodes/execnodes.h           |   8 ++
 src/include/nodes/plannodes.h           |   6 +
 src/include/nodes/relation.h            |  12 ++
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   4 +-
 20 files changed, 564 insertions(+), 50 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f07ac15..38d57a7 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,6 +34,7 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -272,11 +273,13 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+									int rt_index, List *clauses,
+									ParamListInfo prmList, ExprContext *exontext);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses, ParamListInfo prmList,
+								 ExprContext *econtext);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -286,7 +289,8 @@ static bool partition_cmp_args(PartitionKey key, int partattoff,
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, Datum *value, ParamListInfo prmList,
+						ExprContext *econtext);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 
@@ -1659,6 +1663,34 @@ get_partition_qual_relid(Oid relid)
 	return result;
 }
 
+void
+get_leaf_part_recurse(Relation rel, List **leaf_part_oids)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+	int			i;
+
+	check_stack_depth();
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		Oid			partrelid = partdesc->oids[i];
+
+		if (get_rel_relkind(partrelid) != RELKIND_PARTITIONED_TABLE)
+			*leaf_part_oids = lappend_oid(*leaf_part_oids, partrelid);
+		else
+		{
+			/*
+			 * We assume all tables in the partition tree were already locked
+			 * by the caller.
+			 */
+			Relation	partrel = heap_open(partrelid, NoLock);
+
+			get_leaf_part_recurse(partrel, leaf_part_oids);
+			heap_close(partrel, NoLock);
+		}
+	}
+}
+
 /*
  * get_partitions_from_clauses
  *		Determine the set of partitions of relation that will satisfy all
@@ -1669,7 +1701,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							List *partclauses, ParamListInfo prmList,
+							ExprContext *econtext)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1703,7 +1736,7 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 	}
 
 	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+												 partclauses, prmList, econtext);
 
 	return result;
 }
@@ -1719,7 +1752,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
-									List *clauses)
+									List *clauses, ParamListInfo prmList,
+									ExprContext *econtext)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
 	Bitmapset *result = NULL;
@@ -1735,7 +1769,7 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, prmList, econtext);
 
 	/*
 	 * The analysis of the matched clauses done by
@@ -1797,7 +1831,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 
 			arg_partset = get_partitions_from_clauses_recurse(relation,
 															  rt_index,
-															  arg_clauses);
+															  arg_clauses,
+															  prmList,
+															  econtext);
 
 			/*
 			 * Partition sets obtained from mutually-disjunctive clauses are
@@ -1853,7 +1889,8 @@ static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, ParamListInfo prmList,
+								 ExprContext *econtext)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -1893,6 +1930,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				continue;
 			}
 		}
+		else if (IsA(lfirst(lc), ExprState))
+			clause = ((ExprState *) lfirst(lc))->expr;
 		else
 			clause = (Expr *) lfirst(lc);
 
@@ -2305,7 +2344,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
-										&keys->maxkeys[i]))
+										&keys->maxkeys[i], prmList, econtext))
 			{
 				keys->n_maxkeys++;
 				keys->max_incl = incl;
@@ -2317,12 +2356,14 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->eqkeys[i]))
+											&keys->eqkeys[i], prmList,
+											econtext))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->maxkeys[i]))
+											&keys->maxkeys[i], prmList,
+											econtext))
 				{
 					keys->n_maxkeys++;
 					keys->max_incl = true;
@@ -2330,7 +2371,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->minkeys[i]))
+											&keys->minkeys[i], prmList,
+											econtext))
 				{
 					keys->n_minkeys++;
 					keys->min_incl = true;
@@ -2338,7 +2380,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
-											 &keys->minkeys[i]))
+											 &keys->minkeys[i], prmList,
+											 econtext))
 			{
 				keys->n_minkeys++;
 				keys->min_incl = incl;
@@ -2426,7 +2469,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, Datum *value, ParamListInfo prmList,
+						ExprContext *econtext)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2467,6 +2511,42 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmList)
+					{
+						Node	   *n = eval_const_expressions_from_list(prmList, (Node *) expr);
+
+						if (IsA(n, Const))
+						{
+							*value = ((Const *) n)->constvalue;
+							return true;
+						}
+					}
+					return false;
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
+			return false;
+
 		default:
 			return false;
 	}
@@ -2728,10 +2808,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, &leftarg_const,
+								 NULL, NULL))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, &rightarg_const,
+								 NULL, NULL))
 		return false;
 
 	/*
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index d275cef..187d30f 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -23,8 +23,6 @@
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
-static PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
-								 int *num_parted, List **leaf_part_oids);
 static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -275,7 +273,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
  * All the relations in the partition tree (including 'rel') must have been
  * locked (using at least the AccessShareLock) by the caller.
  */
-static PartitionDispatch *
+PartitionDispatch *
 RelationGetPartitionDispatchInfo(Relation rel,
 								 int *num_parted, List **leaf_part_oids)
 {
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..0254b99 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,13 @@
 
 #include "postgres.h"
 
+#include "catalog/pg_inherits_fn.h"
+#include "nodes/relation.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
+#include "executor/execPartition.h"
 #include "miscadmin.h"
+#include "optimizer/clauses.h"
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool exec_append_initialize_next(AppendState *appendstate);
@@ -107,6 +111,24 @@ exec_append_initialize_next(AppendState *appendstate)
 	}
 }
 
+static List *
+initClauses(PlanState *parent, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		new_list = lappend(new_list, ExecInitExpr(val, parent));
+	}
+	return new_list;
+}
+
 /* ----------------------------------------------------------------
  *		ExecInitAppend
  *
@@ -151,6 +173,11 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->index = NIL;
+	appendstate->as_whichpartition = -1;
+	appendstate->array_size = node->array_size;
+	appendstate->append_paths_array = node->append_paths_array;
+	appendstate->parentoid = node->parentoid;
 
 	/*
 	 * Miscellaneous initialization
@@ -164,6 +191,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 * we have to initialize them.
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
@@ -178,9 +206,16 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		i++;
 	}
 
-	/*
-	 * initialize output tuple type
-	 */
+	if (node->extern_quals)
+	{
+		appendstate->extern_quals = node->extern_quals;
+		appendstate->es_param_list_info = estate->es_param_list_info;
+	}
+
+	if (node->join_clauses)
+		appendstate->join_clauses = initClauses((PlanState *) appendstate,
+												node->join_clauses);
+
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
@@ -204,6 +239,85 @@ ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
 
+	if (node->extern_quals && node->as_whichpartition == -1)
+	{
+		Relation	rel;
+		Bitmapset  *partset = NULL;
+		Bitmapset  *subplan_indexes = NULL;
+		int			i,
+					cur_index;
+		PartitionDispatch *pd,
+				   *p1,
+					parent;
+		int			num_parted;
+		List	   *leaf_part_oids = NIL;
+		List	   *parents = NIL;
+
+		rel = relation_open(node->parentoid, NoLock);
+
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		(void) find_all_inheritors(RelationGetRelid(rel), AccessShareLock, NULL);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		relation_close(rel, NoLock);
+		parents = lappend(parents, &pd[0]);
+		do
+		{
+			p1 = linitial(parents);
+			parent = *p1;
+
+			partset = get_partitions_from_clauses(parent->reldesc, 1,
+												  list_copy(node->extern_quals),
+												  node->es_param_list_info,
+												  node->ps.ps_ExprContext);
+
+			if (!bms_is_empty(partset))
+			{
+				i = 0;
+				while ((cur_index = bms_first_member(partset)) >= 0)
+				{
+					if (parent->indexes[cur_index] >= 0)
+						subplan_indexes =
+							bms_add_member(subplan_indexes, parent->indexes[cur_index]);
+					else
+						parents =
+							lappend(parents, &pd[-parent->indexes[cur_index]]);
+				}
+			}
+
+			parents = list_delete_first(parents);
+		} while (parents);
+
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+
+		if (!bms_is_empty(subplan_indexes))
+		{
+			int			j;
+
+			while ((j = bms_first_member(subplan_indexes)) >= 0)
+			{
+				int			index = node->append_paths_array[j];
+
+				if (index >= 0)
+					node->index = lappend_int(node->index, index);
+			}
+			if (node->index)
+			{
+				node->as_whichplan = linitial_int(node->index);
+				node->as_whichpartition = 0;
+			}
+			else
+				node->as_whichplan = 0;
+		}
+	}
 	for (;;)
 	{
 		PlanState  *subnode;
@@ -237,7 +351,22 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->index)
+			{
+				if (++node->as_whichpartition < list_length(node->index))
+					node->as_whichplan = list_nth_int(node->index,
+													  node->as_whichpartition);
+				else
+					return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -280,6 +409,79 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	if (node->ps.chgParam != NULL && node->join_clauses)
+	{
+		Bitmapset  *partset = NULL,
+				   *subplan_indexes = NULL;
+		Relation	rel;
+		PartitionDispatch *pd,
+					parent,
+				   *p1;
+		List	   *parents = NIL;
+		List	   *leaf_part_oids = NIL;
+		int			cur_index,
+					num_parted;
+
+		rel = relation_open(node->parentoid, NoLock);
+
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		(void) find_all_inheritors(RelationGetRelid(rel), AccessShareLock, NULL);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		relation_close(rel, NoLock);
+		parents = lappend(parents, &pd[0]);
+		node->index = NIL;
+		do
+		{
+			p1 = linitial(parents);
+			parent = *p1;
+
+			partset = get_partitions_from_clauses(parent->reldesc,
+												  1,
+												  list_copy(node->join_clauses),
+												  NULL,
+												  node->ps.ps_ExprContext);
+
+			if (!bms_is_empty(partset))
+			{
+				i = 0;
+				while ((cur_index = bms_first_member(partset)) >= 0)
+				{
+					if (parent->indexes[cur_index] >= 0)
+						subplan_indexes =
+							bms_add_member(subplan_indexes, parent->indexes[cur_index]);
+					else
+						parents =
+							lappend(parents, &pd[-parent->indexes[cur_index]]);
+				}
+			}
+			parents = list_delete_first(parents);
+		} while (parents);
+
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+
+		if (!bms_is_empty(subplan_indexes))
+		{
+			int			j;
+
+			while ((j = bms_first_member(subplan_indexes)) >= 0)
+			{
+				int			index = node->append_paths_array[j];
+
+				if (index >= 0)
+					node->index = lappend_int(node->index, index);
+			}
+		}
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +500,13 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->index)
+	{
+		node->as_whichplan = linitial_int(node->index);
+		node->as_whichpartition = 0;
+	}
+	else
+		node->as_whichplan = 0;
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d9ff8a7..74408ae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,11 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(extern_quals);
+	COPY_NODE_FIELD(join_clauses);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_POINTER_FIELD(append_paths_array, from->array_size * sizeof(int));
+	COPY_SCALAR_FIELD(array_size);
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6b087ec..98fa866 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -26,6 +26,7 @@
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_type.h"
+#include "catalog/pg_inherits_fn.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -140,13 +141,15 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 						List *live_childrels);
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte);
+						  RangeTblEntry *rte,
+						  List *chk_clauses);
 static List *match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse);
-
+static int	list_member_oid_index(List *list, Oid datum);
 
 /*
  * make_one_rel
@@ -289,6 +292,22 @@ set_base_rel_sizes(PlannerInfo *root)
 		if (root->glob->parallelModeOK)
 			set_rel_consider_parallel(root, rel, rte);
 
+		if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			List	   *leaf_parts = NIL;
+			Relation	parent = relation_open(rte->relid, NoLock);
+			int			i;
+
+			(void) find_all_inheritors(RelationGetRelid(parent), AccessShareLock, NULL);
+			get_leaf_part_recurse(parent, &leaf_parts);
+			root->array_size = list_length(leaf_parts);
+			root->leaf_node_oids = leaf_parts;
+			root->append_paths_array = palloc0(root->array_size * sizeof(int));
+			for (i = 0; i < root->array_size; i++)
+				root->append_paths_array[i] = -1;
+			root->append_count = 0;
+			relation_close(parent, NoLock);
+		}
 		set_rel_size(root, rel, rti, rte);
 	}
 }
@@ -350,6 +369,41 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	{
 		/* It's an "append relation", process accordingly */
 		set_append_rel_size(root, rel, rti, rte);
+
+		if (rel->joininfo && rel->part_scheme)
+		{
+			List	   *partclauses;
+			bool		contains_param,
+						contains_const,
+						constfalse;
+
+			/*
+			 * Get the clauses that match the partition key, including
+			 * information about any nullness tests against partition keys.
+			 * Set keynullness to a invalid value of NullTestType, which 0 is
+			 * not.
+			 */
+			partclauses = match_clauses_to_partkey(root, rel,
+												   list_copy(rel->joininfo),
+												   &contains_param,
+												   &contains_const,
+												   &constfalse);
+
+			if (partclauses != NIL)
+			{
+				ListCell   *lc;
+
+				foreach(lc, partclauses)
+				{
+					Node	   *n = lfirst(lc);
+
+					if (!list_member(root->join_append_clauses, n));
+					root->join_append_clauses = lappend(root->join_append_clauses, n);
+				}
+			}
+
+		}
+
 	}
 	else
 	{
@@ -867,11 +921,13 @@ set_foreign_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
 static List *
 get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
-						  RangeTblEntry *rte)
+						  RangeTblEntry *rte,
+						  List *chk_clauses)
 {
 	List   *partclauses;
 	bool	contains_const,
-			constfalse;
+			constfalse,
+			contains_param;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -880,6 +936,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &contains_param,
 										   &contains_const,
 										   &constfalse);
 
@@ -897,7 +954,7 @@ get_append_rel_partitions(PlannerInfo *root,
 		 */
 		if (partclauses != NIL && contains_const)
 			partindexes = get_partitions_from_clauses(parent, rel->relid,
-													  partclauses);
+													  partclauses, NULL, NULL);
 
 		/*
 		 * Else there are no clauses that are useful to prune any paritions,
@@ -906,6 +963,18 @@ get_append_rel_partitions(PlannerInfo *root,
 		else
 			partindexes = bms_add_range(NULL, 0, partdesc->nparts - 1);
 
+		if (partclauses != NIL && contains_param)
+		{
+			ListCell   *lc;
+
+			foreach(lc, partclauses)
+			{
+				Node	   *n = lfirst(lc);
+
+				if (!list_member(root->extern_clauses, n));
+				root->extern_clauses = lappend(root->extern_clauses, n);
+			}
+		}
 		/* Fetch the partition appinfos. */
 		i = -1;
 		while ((i = bms_next_member(partindexes, i)) >= 0)
@@ -964,11 +1033,14 @@ get_append_rel_partitions(PlannerInfo *root,
  *
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
+ *
+ * If the list has a param, *contains_param is set
  */
 static List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1028,10 +1100,12 @@ match_clauses_to_partkey(PlannerInfo *root,
 				foreach (lc1, orclause->args)
 				{
 					Node   *arg = lfirst(lc1);
-					bool	contains_const1,
-							constfalse1;
+					bool	contains_const1;
+					bool	constfalse1;
+					bool 	contains_param1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 &contains_param1,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1150,6 +1224,9 @@ match_clauses_to_partkey(PlannerInfo *root,
 				if (contain_volatile_functions((Node *) constexpr))
 					continue;
 
+				if (IsA(constexpr, Param))
+					*contains_param = true;
+
 				/*
 				 * Everything seems to be fine, so add it to the list of
 				 * clauses we will use for pruning.
@@ -1249,6 +1326,23 @@ match_clauses_to_partkey(PlannerInfo *root,
 	return result;
 }
 
+static int
+list_member_oid_index(List *list, Oid datum)
+{
+	int			i = 0;
+	const ListCell *cell;
+
+	foreach(cell, list)
+	{
+		if (lfirst_oid(cell) == datum)
+			return i;
+		i++;
+	}
+
+	return -1;
+
+}
+
 /*
  * set_append_rel_size
  *	  Set size estimates for a simple "append relation"
@@ -1264,7 +1358,8 @@ static void
 set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 					Index rti, RangeTblEntry *rte)
 {
-	List	   *rel_appinfos = NIL;
+	List	   *rel_appinfos = NIL,
+			   *chk_clauses = NIL;
 	int			parentRTindex = rti;
 	bool		has_live_children;
 	double		parent_rows;
@@ -1280,7 +1375,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 
 	if (rte->relkind != RELKIND_PARTITIONED_TABLE)
 	{
-		foreach (l, root->append_rel_list)
+		foreach(l, root->append_rel_list)
 		{
 			AppendRelInfo   *appinfo = lfirst(l);
 
@@ -1291,7 +1386,7 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	}
 	else
 	{
-		rel_appinfos = get_append_rel_partitions(root, rel, rte);
+		rel_appinfos = get_append_rel_partitions(root, rel, rte, chk_clauses);
 		rel->live_partitioned_rels = list_make1_int(rti);
 	}
 
@@ -1641,7 +1736,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, rel_appinfos)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
-		int			childRTindex;
+		int			childRTindex,
+					index;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 
@@ -1674,6 +1770,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* Only consider non dummy children */
+		index = list_member_oid_index(root->leaf_node_oids, childRTE->relid);
+		if (index >= 0)
+			root->append_paths_array[index] = root->append_count++;
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
@@ -1855,7 +1956,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1882,7 +1983,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1936,7 +2037,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2173,7 +2274,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index b491fb9..61fe647 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d445477..91a2bdb 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,24 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static List *
+replace_partition_nestloop_params(PlannerInfo *root, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Node	   *n = lfirst(lc);
+
+		new_list = lappend(new_list, replace_nestloop_params(root, n));
+	}
+	return new_list;
+}
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1081,12 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	plan->extern_quals = best_path->extern_quals;
+	plan->join_clauses = replace_partition_nestloop_params(root, best_path->join_clauses);
+	plan->parentoid = best_path->parentoid;
+	plan->array_size = best_path->array_size;
+	plan->append_paths_array = best_path->append_paths_array;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 28093ac..ffefcd0 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 72f1fa3..45e3598 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -5217,3 +5217,17 @@ tlist_matches_coltypelist(List *tlist, List *coltypelist)
 
 	return true;
 }
+
+Node *
+eval_const_expressions_from_list(ParamListInfo prmList, Node *node)
+{
+	eval_const_expressions_context context;
+
+	if (prmList)
+		context.boundParams = prmList;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 68dee0f..4a19e76 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,34 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	if (root)
+	{
+		pathnode->append_paths_array = root->append_paths_array;
+		pathnode->array_size = root->array_size;
+
+	}
+	if (root && (required_outer || root->extern_clauses))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				if (root->join_append_clauses)
+					pathnode->join_clauses = root->join_append_clauses;
+
+				if (root->extern_clauses)
+					pathnode->extern_quals = root->extern_clauses;
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index a968fa4..495b64b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1555,6 +1555,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1566,11 +1569,29 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+
+	/*
+	 * Generally for appendrel we don't fetch the clause from the join clause
+	 * (only we do so for baserel), but for identifying whether the appendrel
+	 * is applicable for runtime pruning or not.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 853c1f6..9656fa4 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1041,7 +1041,7 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	if (plansource->num_custom_plans < 5)
 		return true;
 
-	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans;
+	avg_custom_cost = plansource->total_custom_cost / plansource->num_custom_plans + 90000;
 
 	/*
 	 * Prefer generic plan if it's less expensive than the average custom
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2041de5..3102f39 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,6 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses);
+							List *partclauses, ParamListInfo prmList, ExprContext *econtext);
+extern void get_leaf_part_recurse(Relation rel, List **leaf_part_oids);
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 64e5aab..6e7a841 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -49,6 +49,9 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
+								 int *num_parted, List **leaf_part_oids);
+
 extern void ExecSetupPartitionTupleRouting(Relation rel,
 							   Index resultRTindex,
 							   EState *estate,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..68b8d35 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,14 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Oid			parentoid;
+	List	   *index;			/* subplan indexes to scan for runtime pruning */
+	int			as_whichpartition;	/* current partition scanned from list */
+	List	   *join_clauses;
+	List	   *extern_quals;
+	ParamListInfo	es_param_list_info;
+	int		   *append_paths_array;
+	int			array_size;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 9b38d44..1908c89 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,11 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+	Oid			parentoid;
+	List	   *extern_quals;
+	List	   *join_clauses;
+	int		   *append_paths_array;
+	int 		array_size;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 94c2e8d..a72e92a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -168,6 +168,7 @@ typedef struct PlannerInfo
 	 */
 	List	   *plan_params;	/* list of PlannerParamItems, see below */
 	Bitmapset  *outer_params;
+	List	   *extern_clauses;
 
 	/*
 	 * simple_rel_array holds pointers to "base rels" and "other rels" (see
@@ -317,6 +318,12 @@ typedef struct PlannerInfo
 
 	/* optional private data for join_search_hook, e.g., GEQO */
 	void	   *join_search_private;
+
+	List	   *leaf_node_oids;
+	List	   *join_append_clauses;
+	int			array_size;
+	int			append_count;
+	int		   *append_paths_array;
 } PlannerInfo;
 
 
@@ -1289,6 +1296,11 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+	List	   *extern_quals;
+	List	   *join_clauses;
+	int		   *append_paths_array;
+	int			array_size;
+	Oid			parentoid;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..c09044e 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prm_list, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..7c1fbfd 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,8 +63,8 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
+				   List *subpaths, Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
-- 
1.8.3.1

#22Robert Haas
robertmhaas@gmail.com
In reply to: Beena Emerson (#21)
Re: [HACKERS] Runtime Partition Pruning

On Fri, Dec 1, 2017 at 6:20 AM, Beena Emerson <memissemerson@gmail.com> wrote:

David Q1:
postgres=# explain analyse execute ab_q1 (3,3); --const
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8) (actual time=0.006..0.006
rows=0 loops=1)
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.005..0.005 rows=0 loops=1)
Filter: ((a = 3) AND (b = 3))
Planning time: 0.588 ms
Execution time: 0.043 ms
(5 rows)

I think the EXPLAIN ANALYZE input should show something attached to
the Append node so that we can tell that partition pruning is in use.
I'm not sure if that is as simple as "Run-Time Partition Pruning: Yes"
or if we can give a few more useful details.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#23Beena Emerson
memissemerson@gmail.com
In reply to: Robert Haas (#22)
Re: [HACKERS] Runtime Partition Pruning

Hello Robert,

On Sat, Dec 2, 2017 at 12:34 AM, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Dec 1, 2017 at 6:20 AM, Beena Emerson <memissemerson@gmail.com> wrote:

David Q1:
postgres=# explain analyse execute ab_q1 (3,3); --const
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8) (actual time=0.006..0.006
rows=0 loops=1)
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.005..0.005 rows=0 loops=1)
Filter: ((a = 3) AND (b = 3))
Planning time: 0.588 ms
Execution time: 0.043 ms
(5 rows)

I think the EXPLAIN ANALYZE input should show something attached to
the Append node so that we can tell that partition pruning is in use.
I'm not sure if that is as simple as "Run-Time Partition Pruning: Yes"
or if we can give a few more useful details.

The output above is shown for a Const Value i.e. optimizer pruning
which I included just to show that the correct partition is chosen
even during runtime pruning for the given value. So taking your
suggestion, the output for runtime pruning could be something as
follows:

postgres=# explain analyse execute ab_q1 (3,3);
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..395.10 rows=9 width=8) (actual time=0.119..0.119
rows=0 loops=1) (run-time partition pruning: on)
-> Seq Scan on ab_a1_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.006..0.006 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.828 ms
Execution time: 0.234 ms
(21 rows)

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#24Robert Haas
robertmhaas@gmail.com
In reply to: Beena Emerson (#23)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Dec 2, 2017 at 3:33 AM, Beena Emerson <memissemerson@gmail.com> wrote:

Append (cost=0.00..395.10 rows=9 width=8) (actual time=0.119..0.119
rows=0 loops=1) (run-time partition pruning: on)

If we can, it would be better to show something a bit more precise,
like the table being used for run-time pruning, or the expression
being used for pruning.

Also, we shouldn't use an ad-hoc format like "(run-time
partition-pruning: on)"; rather, we should display something using one
of the ExplainPropertyBlah functions in explain.c.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#25David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#22)
Re: [HACKERS] Runtime Partition Pruning

On 2 December 2017 at 08:04, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Dec 1, 2017 at 6:20 AM, Beena Emerson <memissemerson@gmail.com> wrote:

David Q1:
postgres=# explain analyse execute ab_q1 (3,3); --const
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8) (actual time=0.006..0.006
rows=0 loops=1)
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.005..0.005 rows=0 loops=1)
Filter: ((a = 3) AND (b = 3))
Planning time: 0.588 ms
Execution time: 0.043 ms
(5 rows)

I think the EXPLAIN ANALYZE input should show something attached to
the Append node so that we can tell that partition pruning is in use.
I'm not sure if that is as simple as "Run-Time Partition Pruning: Yes"
or if we can give a few more useful details.

It already does. Anything subnode with "(never executed)" was pruned
at runtime. Do we really need anything else to tell us that?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#26Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#25)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Dec 6, 2017 at 1:21 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 2 December 2017 at 08:04, Robert Haas <robertmhaas@gmail.com> wrote:

On Fri, Dec 1, 2017 at 6:20 AM, Beena Emerson <memissemerson@gmail.com> wrote:

David Q1:
postgres=# explain analyse execute ab_q1 (3,3); --const
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..43.90 rows=1 width=8) (actual time=0.006..0.006
rows=0 loops=1)
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.005..0.005 rows=0 loops=1)
Filter: ((a = 3) AND (b = 3))
Planning time: 0.588 ms
Execution time: 0.043 ms
(5 rows)

I think the EXPLAIN ANALYZE input should show something attached to
the Append node so that we can tell that partition pruning is in use.
I'm not sure if that is as simple as "Run-Time Partition Pruning: Yes"
or if we can give a few more useful details.

It already does. Anything subnode with "(never executed)" was pruned
at runtime. Do we really need anything else to tell us that?

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

Few cases:

1. Only runtime pruning - David's case1
explain analyse execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..395.10 rows=9 width=8) (actual time=0.101..0.101
rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.007..0.007 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.780 ms
Execution time: 0.220 ms
(22 rows)

2. Runtime pruning after optimizer pruning - David's case 2.
((a >= 4) AND (a <= 5) is used during optimizer pruning and only (a =
$1) is used for runtime pruning.
=# explain (analyse, costs off, summary off) execute ab_q1 (4);
QUERY PLAN
-------------------------------------------------------------------
Append (actual time=0.062..0.062 rows=0 loops=1)
Runtime Partition Pruning: (a = $1)
-> Seq Scan on ab_a4 (actual time=0.005..0.005 rows=0 loops=1)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
-> Seq Scan on ab_a5 (never executed)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
(6 rows)

3. Nestloop Join
tbl1.col1 only has values from 1 to 10.

=# \d+ tprt
Table "public.tprt"
Column | Type | Collation | Nullable | Default | Storage | Stats
target | Description
--------+---------+-----------+----------+---------+---------+--------------+-------------
col1 | integer | | | | plain | |
col2 | integer | | | | plain | |
Partition key: RANGE (col1)
Partitions: tprt_1 FOR VALUES FROM (1) TO (5001),
tprt_2 FOR VALUES FROM (5001) TO (10001),
tprt_3 FOR VALUES FROM (10001) TO (20001)

=# explain (analyse, costs off, summary off) SELECT * FROM tbl1 JOIN
tprt ON tbl1.col1 > tprt.col1;
QUERY PLAN
--------------------------------------------------------------------------------------------
Nested Loop (actual time=0.053..0.192 rows=45 loops=1)
-> Seq Scan on tbl1 (actual time=0.007..0.009 rows=10 loops=1)
-> Append (actual time=0.003..0.004 rows=4 loops=10)
Runtime Partition Pruning Join Filter: (tbl1.col1 > col1)
-> Index Scan using tprt1_idx on tprt_1 (actual
time=0.002..0.004 rows=5 loops=9)
Index Cond: (tbl1.col1 > col1)
-> Index Scan using tprt2_idx on tprt_2 (never executed)
Index Cond: (tbl1.col1 > col1)
-> Index Scan using tprt3_idx on tprt_3 (never executed)
Index Cond: (tbl1.col1 > col1)
(10 rows)

4. InitPlan - Raghu's test case:
4.1 Only few partitions satisfy the param
explain (analyse, costs off, summary off) SELECT * FROM prun_test_part
WHERE sal < (SELECT sal FROM prun_test_part WHERE sal = 200);
QUERY PLAN
-----------------------------------------------------------------------------------------------------------
Append (actual time=0.034..0.038 rows=3 loops=1)
Runtime Partition Pruning: (sal < $0)
InitPlan 1 (returns $0)
-> Append (actual time=0.008..0.009 rows=1 loops=1)
-> Seq Scan on prun_test_part_p3 prun_test_part_p3_1
(actual time=0.008..0.009 rows=1 loops=1)
Filter: (sal = 200)
Rows Removed by Filter: 1
-> Seq Scan on prun_test_part_p1 (actual time=0.002..0.003 rows=1 loops=1)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p2 (actual time=0.002..0.003 rows=2 loops=1)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p3 (never executed)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p4 (never executed)
Filter: (sal < $0)
(15 rows)

4.2 When the InitPlan query returns nothing
=# explain (analyse, costs off, summary off) SELECT * FROM
prun_test_part WHERE sal < (SELECT sal FROM prun_test_part WHERE sal =
50);
QUERY PLAN
-----------------------------------------------------------------------------------------------------------
Append (actual time=0.050..0.050 rows=0 loops=1)
Runtime Partition Pruning: (sal < $0)
InitPlan 1 (returns $0)
-> Append (actual time=0.013..0.013 rows=0 loops=1)
-> Seq Scan on prun_test_part_p1 prun_test_part_p1_1
(actual time=0.012..0.012 rows=0 loops=1)
Filter: (sal = 50)
Rows Removed by Filter: 1
-> Seq Scan on prun_test_part_p1 (never executed)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p2 (never executed)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p3 (never executed)
Filter: (sal < $0)
-> Seq Scan on prun_test_part_p4 (never executed)
Filter: (sal < $0)
(15 rows)

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning_v5.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning_v5.patchDownload
From f3884e16e2e1db6cd91961ed37113d89244fb906 Mon Sep 17 00:00:00 2001
From: Beena Emerson <Beena.Emerson@EnterpriseDB.com>
Date: Thu, 7 Dec 2017 10:54:29 +0530
Subject: [PATCH] Implement Runtime Partition Pruning

Patch by: Beena Emerson, Dilip Kumar
Discussion: https://postgr.es/m/CAOG9ApE16ac-_VVZVvv0gePSgkg_BwYEV1NBqZFqDR2bBE0X0A@mail.gmail.com
---
 src/backend/catalog/partition.c         | 128 ++++++++++++++++++---
 src/backend/commands/explain.c          |  15 +++
 src/backend/executor/execPartition.c    |   4 +-
 src/backend/executor/nodeAppend.c       | 192 ++++++++++++++++++++++++++++++--
 src/backend/nodes/copyfuncs.c           |   5 +
 src/backend/optimizer/path/allpaths.c   | 146 ++++++++++++++++++++++--
 src/backend/optimizer/path/joinrels.c   |   2 +-
 src/backend/optimizer/plan/createplan.c |  24 ++++
 src/backend/optimizer/plan/planner.c    |   2 +-
 src/backend/optimizer/prep/prepunion.c  |   4 +-
 src/backend/optimizer/util/clauses.c    |  19 ++++
 src/backend/optimizer/util/pathnode.c   |  52 ++++++++-
 src/backend/optimizer/util/relnode.c    |  21 +++-
 src/backend/utils/cache/plancache.c     |   2 +-
 src/include/catalog/partition.h         |   5 +-
 src/include/executor/execPartition.h    |   3 +
 src/include/nodes/execnodes.h           |  10 ++
 src/include/nodes/plannodes.h           |   8 ++
 src/include/nodes/relation.h            |  18 +++
 src/include/optimizer/clauses.h         |   1 +
 src/include/optimizer/pathnode.h        |   4 +-
 21 files changed, 614 insertions(+), 51 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f07ac15..b4a4965 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,6 +34,7 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -272,11 +273,14 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+									int rt_index, List *clauses,
+									ParamListInfo base_prmlist,
+									ExprContext *exontext);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -286,7 +290,8 @@ static bool partition_cmp_args(PartitionKey key, int partattoff,
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 
@@ -1659,6 +1664,37 @@ get_partition_qual_relid(Oid relid)
 	return result;
 }
 
+/* get_leaf_part_recurse
+ *		Get the leaf oids for the given rel.
+ */
+void
+get_leaf_part_recurse(Relation rel, List **leaf_part_oids)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+	int			i;
+
+	check_stack_depth();
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		Oid			partrelid = partdesc->oids[i];
+
+		if (get_rel_relkind(partrelid) != RELKIND_PARTITIONED_TABLE)
+			*leaf_part_oids = lappend_oid(*leaf_part_oids, partrelid);
+		else
+		{
+			/*
+			 * We assume all tables in the partition tree were already locked
+			 * by the caller.
+			 */
+			Relation	partrel = heap_open(partrelid, NoLock);
+
+			get_leaf_part_recurse(partrel, leaf_part_oids);
+			heap_close(partrel, NoLock);
+		}
+	}
+}
+
 /*
  * get_partitions_from_clauses
  *		Determine the set of partitions of relation that will satisfy all
@@ -1669,7 +1705,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1703,7 +1740,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 	}
 
 	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+												 partclauses, base_prmlist,
+												 econtext);
 
 	return result;
 }
@@ -1719,7 +1757,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
-									List *clauses)
+									List *clauses, ParamListInfo base_prmlist,
+									ExprContext *econtext)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
 	Bitmapset *result = NULL;
@@ -1735,7 +1774,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, base_prmlist,
+											 econtext);
 
 	/*
 	 * The analysis of the matched clauses done by
@@ -1797,7 +1837,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 
 			arg_partset = get_partitions_from_clauses_recurse(relation,
 															  rt_index,
-															  arg_clauses);
+															  arg_clauses,
+															  base_prmlist,
+															  econtext);
 
 			/*
 			 * Partition sets obtained from mutually-disjunctive clauses are
@@ -1853,7 +1895,8 @@ static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -1893,6 +1936,9 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				continue;
 			}
 		}
+		/* when called from ExecReScanAppend */
+		else if (IsA(lfirst(lc), ExprState))
+			clause = ((ExprState *) lfirst(lc))->expr;
 		else
 			clause = (Expr *) lfirst(lc);
 
@@ -2305,7 +2351,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
-										&keys->maxkeys[i]))
+										&keys->maxkeys[i], base_prmlist,
+										econtext))
 			{
 				keys->n_maxkeys++;
 				keys->max_incl = incl;
@@ -2317,12 +2364,14 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->eqkeys[i]))
+											&keys->eqkeys[i], base_prmlist,
+											econtext))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->maxkeys[i]))
+											&keys->maxkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_maxkeys++;
 					keys->max_incl = true;
@@ -2330,7 +2379,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->minkeys[i]))
+											&keys->minkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_minkeys++;
 					keys->min_incl = true;
@@ -2338,7 +2388,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
-											 &keys->minkeys[i]))
+											 &keys->minkeys[i], base_prmlist,
+											 econtext))
 			{
 				keys->n_minkeys++;
 				keys->min_incl = incl;
@@ -2426,7 +2477,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2467,6 +2519,46 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (base_prmlist)
+					{
+						Node	   *n;
+
+						n = eval_const_expressions_from_list(base_prmlist,
+															 (Node *) expr);
+
+						if (IsA(n, Const))
+						{
+							*value = ((Const *) n)->constvalue;
+							return true;
+						}
+					}
+					return false;
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
+			return false;
+
 		default:
 			return false;
 	}
@@ -2728,10 +2820,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, &leftarg_const,
+								 NULL, NULL))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, &rightarg_const,
+								 NULL, NULL))
 		return false;
 
 	/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 447f69d..06c62cb 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1347,6 +1347,21 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	if (es->format == EXPLAIN_FORMAT_TEXT)
 		appendStringInfoChar(es->str, '\n');
 
+	/* run-time pruning information for Append node */
+	if (es->analyze && IsA(plan, Append))
+	{
+		Append	   *append = (Append *) plan;
+
+		if (append->base_params)
+			show_scan_qual(append->base_params, "Runtime Partition Pruning",
+						   planstate, ancestors, es);
+
+		if (append->join_clauses)
+			show_scan_qual(append->join_clauses,
+						   "Runtime Partition Pruning Join Filter",
+						   planstate, ancestors, es);
+	}
+
 	/* target list */
 	if (es->verbose)
 		show_plan_tlist(planstate, ancestors, es);
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index d275cef..187d30f 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -23,8 +23,6 @@
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
-static PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
-								 int *num_parted, List **leaf_part_oids);
 static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -275,7 +273,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
  * All the relations in the partition tree (including 'rel') must have been
  * locked (using at least the AccessShareLock) by the caller.
  */
-static PartitionDispatch *
+PartitionDispatch *
 RelationGetPartitionDispatchInfo(Relation rel,
 								 int *num_parted, List **leaf_part_oids)
 {
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..073d14f 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,7 +57,9 @@
 
 #include "postgres.h"
 
+#include "catalog/pg_inherits_fn.h"
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -108,6 +110,111 @@ exec_append_initialize_next(AppendState *appendstate)
 }
 
 /* ----------------------------------------------------------------
+ *		initClauses
+ *
+ *		Runs ExecInitExpr for each cell in given list.
+ *
+ *		Returns NULL if list is NULL
+ * ----------------------------------------------------------------
+ */
+static List *
+initClauses(PlanState *parent, List *list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (list == NULL)
+		return NULL;
+
+	foreach(lc, list)
+	{
+		Expr	   *val = (Expr *) lfirst(lc);
+
+		new_list = lappend(new_list, ExecInitExpr(val, parent));
+	}
+	return new_list;
+}
+
+/* ----------------------------------------------------------------
+ *		set_append_subplan_indexes
+ *
+ *		Determine the subplans to scan based on the clauses.
+ * ----------------------------------------------------------------
+ */
+static void
+set_append_subplan_indexes(AppendState *node, List *clauses)
+{
+	Bitmapset  *partset = NULL,
+			   *subplans = NULL;
+	Relation	rel;
+	PartitionDispatch *pd,
+				parent,
+			   *p1;
+	List	   *parents = NIL;
+	List	   *leaf_part_oids = NIL;
+	int			cur_index,
+				num_parted,
+				i;
+
+	/*
+	 * Get the information about the partition tree after locking all the
+	 * partitions.
+	 */
+	rel = relation_open(node->parentoid, NoLock);
+	(void) find_all_inheritors(RelationGetRelid(rel), AccessShareLock, NULL);
+	pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+	relation_close(rel, NoLock);
+
+	parents = lappend(parents, &pd[0]);
+	node->subplan_indexes = NIL;
+	do
+	{
+		p1 = linitial(parents);
+		parent = *p1;
+
+		partset = get_partitions_from_clauses(parent->reldesc,
+											  1,
+											  list_copy(clauses),
+											  node->es_param_list_info,
+											  node->ps.ps_ExprContext);
+
+		if (!bms_is_empty(partset))
+		{
+			while ((cur_index = bms_first_member(partset)) >= 0)
+			{
+				if (parent->indexes[cur_index] >= 0)
+					subplans =
+						bms_add_member(subplans, parent->indexes[cur_index]);
+				else
+					parents =
+						lappend(parents, &pd[-parent->indexes[cur_index]]);
+			}
+		}
+		parents = list_delete_first(parents);
+	} while (parents);
+
+	for (i = 1; i < num_parted; i++)
+	{
+		PartitionDispatch partdispatch = pd[i];
+
+		heap_close(partdispatch->reldesc, NoLock);
+		ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+	}
+
+	if (!bms_is_empty(subplans))
+	{
+		while ((i = bms_first_member(subplans)) >= 0)
+		{
+			int			index = node->append_paths_array[i];
+
+			if (index >= 0)
+				node->subplan_indexes = lappend_int(node->subplan_indexes,
+													index);
+		}
+	}
+}
+
+/* ----------------------------------------------------------------
  *		ExecInitAppend
  *
  *		Begin all of the subscans of the append node.
@@ -151,17 +258,22 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->subplan_indexes = NIL;
+	appendstate->as_whichpartition = -1;
+	appendstate->append_paths_size = node->append_paths_size;
+	appendstate->append_paths_array = node->append_paths_array;
+	appendstate->parentoid = node->parentoid;
 
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
 
+	/* create expression context for node */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * append nodes have Result slots, which hold pointers to tuples, so we
+	 * have to initialize them.
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
@@ -178,9 +290,16 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		i++;
 	}
 
-	/*
-	 * initialize output tuple type
-	 */
+	if (node->base_params)
+	{
+		appendstate->base_params = node->base_params;
+		appendstate->es_param_list_info = estate->es_param_list_info;
+	}
+
+	if (node->join_clauses)
+		appendstate->join_clauses = initClauses((PlanState *) appendstate,
+												node->join_clauses);
+
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
@@ -204,6 +323,19 @@ ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
 
+	/* Determine the subplans to scan depending on the base_params */
+	if (node->base_params && node->as_whichpartition == -1)
+	{
+		set_append_subplan_indexes(node, node->base_params);
+		if (node->subplan_indexes)
+		{
+			node->as_whichplan = linitial_int(node->subplan_indexes);
+			node->as_whichpartition = 0;
+		}
+		else
+			node->as_whichplan = 0;
+	}
+
 	for (;;)
 	{
 		PlanState  *subnode;
@@ -212,8 +344,19 @@ ExecAppend(PlanState *pstate)
 		CHECK_FOR_INTERRUPTS();
 
 		/*
+		 * end scan if no subplan is selected for the current join_clauses
+		 * and/or base_params
+		 */
+		if ((node->join_clauses || node->base_params)
+			&& node->as_whichpartition == -1)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/*
 		 * figure out which subplan we are currently processing
 		 */
+		if (node->as_whichpartition != -1)
+			node->as_whichplan = list_nth_int(node->subplan_indexes,
+											  node->as_whichpartition);
 		subnode = node->appendplans[node->as_whichplan];
 
 		/*
@@ -237,7 +380,22 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->subplan_indexes)
+			{
+				if (++node->as_whichpartition < list_length(node->subplan_indexes))
+					node->as_whichplan = list_nth_int(node->subplan_indexes,
+													  node->as_whichpartition);
+				else
+					return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -280,6 +438,10 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/* Determine subplans to scan based on the new Params */
+	if (node->ps.chgParam != NULL && node->join_clauses)
+		set_append_subplan_indexes(node, node->join_clauses);
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +460,16 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->subplan_indexes)
+	{
+		node->as_whichplan = linitial_int(node->subplan_indexes);
+		node->as_whichpartition = 0;
+	}
+	else
+	{
+		node->as_whichplan = 0;
+		node->as_whichpartition = -1;
+	}
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d9ff8a7..4af11e8 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,11 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(base_params);
+	COPY_NODE_FIELD(join_clauses);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_POINTER_FIELD(append_paths_array, from->append_paths_size * sizeof(int));
+	COPY_SCALAR_FIELD(append_paths_size);
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6b087ec..6d75b41 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -26,6 +26,7 @@
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_type.h"
+#include "catalog/pg_inherits_fn.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -144,9 +145,10 @@ static List *get_append_rel_partitions(PlannerInfo *root,
 static List *match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse);
-
+static int	list_member_oid_index(List *list, Oid datum);
 
 /*
  * make_one_rel
@@ -289,6 +291,27 @@ set_base_rel_sizes(PlannerInfo *root)
 		if (root->glob->parallelModeOK)
 			set_rel_consider_parallel(root, rel, rte);
 
+		if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			List	   *leaf_parts = NIL;
+			Relation	parent = relation_open(rte->relid, NoLock);
+			int			i;
+
+			/* fetch the leaf oids of the parent rel */
+			(void) find_all_inheritors(RelationGetRelid(parent), AccessShareLock, NULL);
+			get_leaf_part_recurse(parent, &leaf_parts);
+			relation_close(parent, NoLock);
+
+			/* initialize necessary root variables */
+			root->append_paths_size = list_length(leaf_parts);
+			root->leaf_node_oids = leaf_parts;
+			root->append_paths_array = palloc0(root->append_paths_size * sizeof(int));
+			root->baserestrictinfo_param_indexes = NIL;
+			root->append_paths_count = 0;
+
+			for (i = 0; i < root->append_paths_size; i++)
+				root->append_paths_array[i] = -1;
+		}
 		set_rel_size(root, rel, rti, rte);
 	}
 }
@@ -350,6 +373,40 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	{
 		/* It's an "append relation", process accordingly */
 		set_append_rel_size(root, rel, rti, rte);
+
+		/*
+		 * If this rel is part of a join then collect the necessary join
+		 * clauses required for runtime partition pruning.
+		 */
+		if (rel->joininfo && rel->part_scheme)
+		{
+			List	   *partclauses;
+			bool		contains_param,
+						contains_const,
+						constfalse;
+
+			/*
+			 * Get the clauses that match the partition key
+			 */
+			partclauses = match_clauses_to_partkey(root, rel,
+												   list_copy(rel->joininfo),
+												   &contains_param,
+												   &contains_const,
+												   &constfalse);
+
+			if (partclauses != NIL)
+			{
+				ListCell   *lc;
+
+				foreach(lc, partclauses)
+				{
+					Node	   *n = lfirst(lc);
+
+					if (!list_member(root->join_clauses, n))
+						root->join_clauses = lappend(root->join_clauses, n);
+				}
+			}
+		}
 	}
 	else
 	{
@@ -871,7 +928,8 @@ get_append_rel_partitions(PlannerInfo *root,
 {
 	List   *partclauses;
 	bool	contains_const,
-			constfalse;
+			constfalse,
+			contains_param;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -880,6 +938,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &contains_param,
 										   &contains_const,
 										   &constfalse);
 
@@ -897,7 +956,7 @@ get_append_rel_partitions(PlannerInfo *root,
 		 */
 		if (partclauses != NIL && contains_const)
 			partindexes = get_partitions_from_clauses(parent, rel->relid,
-													  partclauses);
+													  partclauses, NULL, NULL);
 
 		/*
 		 * Else there are no clauses that are useful to prune any paritions,
@@ -964,17 +1022,21 @@ get_append_rel_partitions(PlannerInfo *root,
  *
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
+ *
+ * If the list has a param, *contains_param is set
  */
 static List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse)
 {
 	PartitionScheme	partscheme = rel->part_scheme;
 	List	   *result = NIL;
 	ListCell   *lc;
+	int			clause_index = -1;
 
 	*contains_const = false;
 	*constfalse = false;
@@ -987,6 +1049,8 @@ match_clauses_to_partkey(PlannerInfo *root,
 		Expr   *clause;
 		int		i;
 
+		clause_index++;
+
 		if (IsA(member, RestrictInfo))
 		{
 			RestrictInfo *rinfo = (RestrictInfo *) member;
@@ -1024,14 +1088,24 @@ match_clauses_to_partkey(PlannerInfo *root,
 				bool	arg_matches_key = false,
 						matched_arg_contains_const = false,
 						all_args_constfalse = true;
+				List	   *base_param_indexes;
+
+				/*
+				 * Make a copy of root's baserestrictinfo_param_indexes before
+				 * the recursive call
+				 */
+				base_param_indexes =
+					list_copy(root->baserestrictinfo_param_indexes);
 
 				foreach (lc1, orclause->args)
 				{
 					Node   *arg = lfirst(lc1);
-					bool	contains_const1,
-							constfalse1;
+					bool	contains_const1;
+					bool	constfalse1;
+					bool	contains_param1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 &contains_param1,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1042,8 +1116,22 @@ match_clauses_to_partkey(PlannerInfo *root,
 					/* We got at least one arg that is not constant false. */
 					if (!constfalse1)
 						all_args_constfalse = false;
+
+					/*
+					 * We got at least one arg which is Param, so add the
+					 * current clause_index to the base_param_indexes list.
+					 */
+					if (contains_param1)
+					{
+						*contains_param = true;
+						base_param_indexes =
+							list_append_unique_int(base_param_indexes,
+												   clause_index);
+					}
 				}
 
+				root->baserestrictinfo_param_indexes = base_param_indexes;
+
 				if (arg_matches_key)
 				{
 					result = lappend(result, clause);
@@ -1150,6 +1239,15 @@ match_clauses_to_partkey(PlannerInfo *root,
 				if (contain_volatile_functions((Node *) constexpr))
 					continue;
 
+				if (IsA(constexpr, Param))
+				{
+					*contains_param = true;
+
+					root->baserestrictinfo_param_indexes =
+						list_append_unique_int(root->baserestrictinfo_param_indexes,
+											   clause_index);
+				}
+
 				/*
 				 * Everything seems to be fine, so add it to the list of
 				 * clauses we will use for pruning.
@@ -1249,6 +1347,26 @@ match_clauses_to_partkey(PlannerInfo *root,
 	return result;
 }
 
+/* list_member_oid_index
+ *	  Returns the index of the given datum in the oid list.
+ *
+ *	It returns -1 if the datum is not found.
+ */
+static int
+list_member_oid_index(List *list, Oid datum)
+{
+	int			i = 0;
+	const ListCell *cell;
+
+	foreach(cell, list)
+	{
+		if (lfirst_oid(cell) == datum)
+			return i;
+		i++;
+	}
+	return -1;
+}
+
 /*
  * set_append_rel_size
  *	  Set size estimates for a simple "append relation"
@@ -1641,7 +1759,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, rel_appinfos)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
-		int			childRTindex;
+		int			childRTindex,
+					index;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 
@@ -1674,6 +1793,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* only consider non dummy children */
+		index = list_member_oid_index(root->leaf_node_oids, childRTE->relid);
+		if (index >= 0)
+			root->append_paths_array[index] = root->append_paths_count++;
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
@@ -1855,7 +1979,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1882,7 +2006,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1936,7 +2060,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2173,7 +2297,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index b491fb9..61fe647 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d445477..a47373f 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,24 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static List *
+replace_partition_nestloop_params(PlannerInfo *root, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Node	   *n = lfirst(lc);
+
+		new_list = lappend(new_list, replace_nestloop_params(root, n));
+	}
+	return new_list;
+}
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1081,12 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	plan->base_params = best_path->base_params;
+	plan->join_clauses = replace_partition_nestloop_params(root, best_path->join_clauses);
+	plan->parentoid = best_path->parentoid;
+	plan->append_paths_size = best_path->append_paths_size;
+	plan->append_paths_array = best_path->append_paths_array;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 28093ac..ffefcd0 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 72f1fa3..fbbcae8 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 68dee0f..9f9738f 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,56 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	/* Do necessary evaluation needed for runtime partition pruning. */
+	if (root && (required_outer || root->baserestrictinfo_param_indexes))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		pathnode->append_paths_array = root->append_paths_array;
+		pathnode->append_paths_size = root->append_paths_size;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				if (root->join_clauses)
+					pathnode->join_clauses = root->join_clauses;
+
+				if (root->baserestrictinfo_param_indexes)
+				{
+					ListCell   *lc;
+					List	   *quals = list_copy(rel->baserestrictinfo);
+
+					pathnode->base_params = NIL;
+
+					/*
+					 * collect the clauses from baserestrictinfo for partition
+					 * pruning
+					 */
+					foreach(lc, root->baserestrictinfo_param_indexes)
+					{
+						int			index = lfirst_int(lc);
+						Node	   *member = list_nth(quals, index);
+						Expr	   *clause;
+
+						if (IsA(member, RestrictInfo))
+							clause = ((RestrictInfo *) member)->clause;
+						else
+							clause = (Expr *) member;
+
+						pathnode->base_params = lappend(pathnode->base_params,
+														clause);
+					}
+				}
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index a968fa4..545a34b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1555,6 +1555,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1566,11 +1569,27 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+	/*
+	 * To determine whether the appendrel is applicable for runtime pruning or
+	 * not, we fetch the clause from the join clause.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2041de5..5a8acd9 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,7 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses);
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext);
+extern void get_leaf_part_recurse(Relation rel, List **leaf_part_oids);
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 64e5aab..6e7a841 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -49,6 +49,9 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
+								 int *num_parted, List **leaf_part_oids);
+
 extern void ExecSetupPartitionTupleRouting(Relation rel,
 							   Index resultRTindex,
 							   EState *estate,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..ad9cdac 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,16 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+
+	/* for runtime partition pruning */
+	Oid			parentoid;		/* oid of the parent */
+	int		   *append_paths_array;
+	int			append_paths_size;	/* size of append_paths_array */
+	List	   *base_params;	/* base restrictinfo on partition keys */
+	List	   *join_clauses;	/* join clauses on partiton keys */
+	ParamListInfo es_param_list_info;
+	List	   *subplan_indexes;	/* List of subplan indexes to scan */
+	int			as_whichpartition;	/* current index to scan in index */
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 9b38d44..698322d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,13 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+
+	/* for runtime partition pruning */
+	Oid			parentoid;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	List	   *base_params;
+	List	   *join_clauses;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 94c2e8d..611fb74 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -317,6 +317,17 @@ typedef struct PlannerInfo
 
 	/* optional private data for join_search_hook, e.g., GEQO */
 	void	   *join_search_private;
+
+	/* These fields accumulate data required for runtime partition pruning. */
+	int		   *append_paths_array; /* array for append subplans info */
+	int			append_paths_size;	/* size of append_paths_array */
+	int			append_paths_count; /* current count of append_paths_array */
+	List	   *leaf_node_oids; /* leaf oids of current rel */
+	List	   *baserestrictinfo_param_indexes; /* index from baserestrictinfo
+												 * list with partition related
+												 * clauses */
+	List	   *join_clauses;	/* join clause which are required for
+								 * partition pruning */
 } PlannerInfo;
 
 
@@ -1289,6 +1300,13 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+
+	/* for runtime partition pruning */
+	Oid			parentoid;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	List	   *base_params;
+	List	   *join_clauses;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..f8e7660 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..7c1fbfd 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,8 +63,8 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
+				   List *subpaths, Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
-- 
1.8.3.1

#27David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#26)
Re: [HACKERS] Runtime Partition Pruning

On 7 December 2017 at 20:22, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch.

Hi Beena,

Thanks for updating this.

Can you list the patches which are required for this to apply to
today's master branch?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#28Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#27)
Re: [HACKERS] Runtime Partition Pruning

Hello David,

On Thu, Dec 7, 2017 at 4:07 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 7 December 2017 at 20:22, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch.

Hi Beena,

Thanks for updating this.

Can you list the patches which are required for this to apply to
today's master branch?

Thanks for looking into this.

Currently Amit's v13 patches do not apply on the HEAD and I was
working on 487a0c1518af2f3ae2d05b7fd23d636d687f28f3 which is the last
commit where all Amit's v13 patches applies cleanly.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#29Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#26)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Thu, Dec 7, 2017 at 12:52 PM, Beena Emerson <memissemerson@gmail.com> wrote:

1. Only runtime pruning - David's case1
explain analyse execute ab_q1 (2,3);
QUERY PLAN
---------------------------------------------------------------------------------------------------------
Append (cost=0.00..395.10 rows=9 width=8) (actual time=0.101..0.101
rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a1_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a2_b3 (cost=0.00..43.90 rows=1 width=8) (actual
time=0.007..0.007 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b1 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b2 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on ab_a3_b3 (cost=0.00..43.90 rows=1 width=8) (never executed)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.780 ms
Execution time: 0.220 ms
(22 rows)

2. Runtime pruning after optimizer pruning - David's case 2.
((a >= 4) AND (a <= 5) is used during optimizer pruning and only (a =
$1) is used for runtime pruning.
=# explain (analyse, costs off, summary off) execute ab_q1 (4);
QUERY PLAN
-------------------------------------------------------------------
Append (actual time=0.062..0.062 rows=0 loops=1)
Runtime Partition Pruning: (a = $1)
-> Seq Scan on ab_a4 (actual time=0.005..0.005 rows=0 loops=1)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
-> Seq Scan on ab_a5 (never executed)
Filter: ((a >= 4) AND (a <= 5) AND (a = $1))
(6 rows)

FYI,

The v4 version of the patch accidentally included the
choose_custom_plan hack I had used to force the runtime pruning in the
above cases(1,2), which has been removed in v5. So with only the patch
applied, it would continue to give the output as with the const and
not the Param because the custom plan is preferred over the generic
one. This was pointed out in the initial post of this thread. Just to
compare, I continued using the hack for the tests to show the
behaviour changes.

A different case would need to be used to test the behaviour which
picks the generic plan.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#30David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#28)
Re: [HACKERS] Runtime Partition Pruning

On 7 December 2017 at 23:56, Beena Emerson <memissemerson@gmail.com> wrote:

Currently Amit's v13 patches do not apply on the HEAD and I was
working on 487a0c1518af2f3ae2d05b7fd23d636d687f28f3 which is the last
commit where all Amit's v13 patches applies cleanly.

Thanks.

I was just looking over this and was wondering about the following case:

drop table if exists p;
create table p (a int not null, b int not null) partition by range (a);
create table p1 partition of p for values from (0) to (1000);
create table p2 partition of p for values from (1000) to (2000);
create table p3 partition of p for values from (2000) to (3000);
create table p4 partition of p for values from (3000) to (4000);

create index on p1 (a);
create index on p2 (a);
create index on p3 (a);
create index on p4 (a);

insert into p select x,x from generate_series(1,3999) x;

drop table if exists t;
create table t (a int not null);

insert into t select generate_Series(1,10);

analyze p;

analyze t;

set enable_mergejoin=0;
set enable_hashjoin=0;

explain analyze select * from p inner join t on p.a = t.a;

The patch gives me:

QUERY PLAN
----------------------------------------------------------------------------------------
Nested Loop (actual time=0.032..0.159 rows=10 loops=1)
-> Seq Scan on t (actual time=0.012..0.013 rows=10 loops=1)
-> Append (actual time=0.004..0.013 rows=1 loops=10)
-> Index Scan using p1_a_idx on p1 (actual time=0.004..0.004
rows=1 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p2_a_idx on p2 (actual time=0.003..0.003
rows=0 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p3_a_idx on p3 (actual time=0.002..0.002
rows=0 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p4_a_idx on p4 (actual time=0.003..0.003
rows=0 loops=10)
Index Cond: (a = t.a)
Planning time: 0.472 ms
Execution time: 0.241 ms
(13 rows)

but I expected to get (never executed) for p2, p3 and p4.

The following code makes me think you intend this to work:

@@ -280,6 +438,10 @@ ExecReScanAppend(AppendState *node)
{
int i;

+ /* Determine subplans to scan based on the new Params */
+ if (node->ps.chgParam != NULL && node->join_clauses)
+ set_append_subplan_indexes(node, node->join_clauses);
+

It just does not due to the node->join_clauses being NULL.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#31Robert Haas
robertmhaas@gmail.com
In reply to: Beena Emerson (#26)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Dec 7, 2017 at 2:22 AM, Beena Emerson <memissemerson@gmail.com> wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

- initClauses() seems to be a duplicate of the existing function
ExecInitExprList(), except for the extra NULL test, which isn't
necessary.

- The executor already has a system for making sure that relations get
opened and locked, and it's different from the new one scheme which
set_append_subplan_indexes() implements. Relations should be locked
during the executor initialization phase (i.e. ExecInitAppend) and not
when the first tuple is requested (i.e. ExecAppend). Also, there's
already code to lock both child relations (i.e. the scans of those
relations, see InitScanRelation, ExecInitIndexScan) and non-leaf
partitions (ExecLockNonLeafAppendTables). The call to
find_all_inheritors() will lock all of that same stuff again *plus*
the leaf partitions that were pruned during planning - moreover, if
the Append is rescanned, we'll walk the partitioning structure again
for every rescan. I think RelationGetPartitionDispatchInfo should be
called directly from ExecInitAppend after the existing code to take
locks has been called, and store a pointer to the PartitionDispatch
object in the AppendState for future use.

- I am surprised that set_append_subplan_indexes() needs to worry
about multi-level partitioning directly. I would have thought that
Amit's patch would take care of that, just returning a Bitmapset of
indexes which this function could use directly. It also doesn't seem
like a very good idea to convert the Bitmapset (subplans) into a list
of integers (node->subplan_indexes), as set_append_subplan_indexes()
does at the bottom. The Bitmapset will be a lot more efficient; we
should be able to just iterate over that directly rather than
converting it into a List. Note that a Bitmapset can be created with
a single palloc, but an integer list needs one per list element plus
one for the list itself.

- I don't think it's a good idea for ExecInitAppend to copy so much
information into the appendstate. It copies append_paths_size,
append_paths_array, parentoid, base_params, es_param_list_info,
join_clauses, but it could just as well access them directly via
planstate->plan and planstate->state when they are needed. Maybe you
had some thought that this would be more efficient, but it probably
doesn't save much and it's unlike what we do elsewhere in the
executor.

- A good chunk of the rest of this logic in nodeAppend.c looks like
it's going to conflict heavily with the Parallel Append patch that
just went in. That's going to require some thought. There's no
reason why a parallel-aware Append can't do runtime partition pruning,
but we want to avoid as much overhead as possible when runtime pruning
isn't chosen. In the parallel-aware case, I think we should just try
to jigger things so that the plans we don't need to scan get marked
pa_finished. We don't want to hold pa_lock while doing the pruning,
so probably what should happen is add a new ParallelAppendState member
indicating wither pruning has been done; any process which needs to
choose a subplan and sees that pruning isn't done yet releases the
lock, performs pruning, then reacquires the lock, marks pa_finished on
all plans that we don't need to scan, marks pruning down, picks a
plan, and releases the lock. There is a race condition where pruning
gets conducted by multiple workers around the same time, but it often
won't happen and isn't a big deal if it does; they should all get the
same answer. In the non-parallel-aware case, I think we should
probably replace choose_next_subplan_locally with
choose_next_subplan_simply (for the non-pruning case) and
choose_next_subplan_with_pruning (for the other case).

- On the optimizer side of things, the new calls to
find_all_inheritors() and get_leaf_part_recurse() in
set_base_rel_sizes() don't look good. As in the executor stuff,
that's work that the optimizer is already doing elsewhere, and we
don't want to redo it here. In the case of the optimizer, the most
relevant place is probably expand_partitioned_rtentry(). Another
place where we already have the relation open is get_relation_info().
Any information you want to get from the relation descriptor needs to
be saved in one of those functions; don't re-open the relation
elsewhere.

- I think you are perhaps doing the work a little too early here,
especially with regards to the join clauses. In set_base_rel_sizes()
we don't have a clear idea what the join order will be, or what type
of join will be used, so we don't know what join clauses are relevant
for run-time pruning. I don't think that trying to identify join
clauses at that stage makes sense. I think that the time when we know
(or can figure out) that stuff might be when we go to build a
parameterized append path - see the bottom of
add_paths_to_append_rel(). It's really the parameterization -- the
relids listed in required_outer -- that tell us which join clauses we
could potentially use for runtime pruning. I'm not sure in detail how
this should work, but it seems to me that generally the right formula
is probably: useful clauses from the appendrel's baserestrictinfo PLUS
joinclauses that mention only vars from the relation itself and
whatever's in required_outer.

- You can't use random fields in PlannerInfo *root as scratch space
for functions in different files to communicate with each other. I
mean, there's obviously some stuff that's valid to stick into the
PlannerInfo, but these are basically per-relation details which you're
counting on clearing before anybody gets confused. That's not a good
design, and it's probably a sign that not all of the code is in the
right place yet. For example, a lot of the logic you've added to
create_append_path() probably belongs in the caller -- look at how
simple create_.*_path() functions generally are. Similarly, think
about whether the chunk of logic added to set_rel_size() just after
the call to set_append_rel_size() doesn't really belong inside that
function, or conversely whether the chunk of code in
set_base_rel_sizes() just before the call to set_rel_size() shuldn't
be moved down. I'm hopeful that with some rejiggering of this sort
you can get the code that needs to communicate closer to being all in
one place and pass around whatever is needed via the parameter lists
of the functions involved, or even local variables, rather than via
PlannerInfo.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#32Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: Beena Emerson (#26)
Re: [HACKERS] Runtime Partition Pruning

Hi Beena,

On 12/07/2017 02:22 AM, Beena Emerson wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

As the current patch conflicts with [1]/messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp could you provide a rebased
version ?

Thanks in advance !

[1]: /messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp
/messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp

Best regards,
Jesper

#33Beena Emerson
memissemerson@gmail.com
In reply to: Robert Haas (#31)
Re: [HACKERS] Runtime Partition Pruning

Hello Robert,
Thank you for the comments. I have started working on it.

On Fri, Dec 8, 2017 at 9:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Dec 7, 2017 at 2:22 AM, Beena Emerson <memissemerson@gmail.com> wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

- initClauses() seems to be a duplicate of the existing function
ExecInitExprList(), except for the extra NULL test, which isn't
necessary.

The initClauses has been removed and ExecInitExprList has been used.

- The executor already has a system for making sure that relations get
opened and locked, and it's different from the new one scheme which
set_append_subplan_indexes() implements. Relations should be locked
during the executor initialization phase (i.e. ExecInitAppend) and not
when the first tuple is requested (i.e. ExecAppend). Also, there's
already code to lock both child relations (i.e. the scans of those
relations, see InitScanRelation, ExecInitIndexScan) and non-leaf
partitions (ExecLockNonLeafAppendTables). The call to
find_all_inheritors() will lock all of that same stuff again *plus*
the leaf partitions that were pruned during planning - moreover, if
the Append is rescanned, we'll walk the partitioning structure again
for every rescan. I think RelationGetPartitionDispatchInfo should be
called directly from ExecInitAppend after the existing code to take
locks has been called, and store a pointer to the PartitionDispatch
object in the AppendState for future use.

I have moved the call to ExecInitAppend. This still uses the previous
locking method, I will work on it in the next version of the patch.

- I am surprised that set_append_subplan_indexes() needs to worry
about multi-level partitioning directly. I would have thought that
Amit's patch would take care of that, just returning a Bitmapset of
indexes which this function could use directly. It also doesn't seem
like a very good idea to convert the Bitmapset (subplans) into a list
of integers (node->subplan_indexes), as set_append_subplan_indexes()
does at the bottom. The Bitmapset will be a lot more efficient; we
should be able to just iterate over that directly rather than
converting it into a List. Note that a Bitmapset can be created with
a single palloc, but an integer list needs one per list element plus
one for the list itself.

The function get_partitions_from_clauses returns the Bitmap set of
partitions for a level of partition. So when the BitmapSet that
indicates a child partitioned table, set_append_subplan_indexes loops
throgh again till it gets the list of all leaf indexes.

I am working on the other comments and will post the patch along with
rebasing to v14 of Amit's patch soon.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#34Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#33)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On Tue, Dec 12, 2017 at 4:57 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello Robert,
Thank you for the comments. I have started working on it.

On Fri, Dec 8, 2017 at 9:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Dec 7, 2017 at 2:22 AM, Beena Emerson <memissemerson@gmail.com> wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

- initClauses() seems to be a duplicate of the existing function
ExecInitExprList(), except for the extra NULL test, which isn't
necessary.

The initClauses has been removed and ExecInitExprList has been used.

- The executor already has a system for making sure that relations get
opened and locked, and it's different from the new one scheme which
set_append_subplan_indexes() implements. Relations should be locked
during the executor initialization phase (i.e. ExecInitAppend) and not
when the first tuple is requested (i.e. ExecAppend). Also, there's
already code to lock both child relations (i.e. the scans of those
relations, see InitScanRelation, ExecInitIndexScan) and non-leaf
partitions (ExecLockNonLeafAppendTables). The call to
find_all_inheritors() will lock all of that same stuff again *plus*
the leaf partitions that were pruned during planning - moreover, if
the Append is rescanned, we'll walk the partitioning structure again
for every rescan. I think RelationGetPartitionDispatchInfo should be
called directly from ExecInitAppend after the existing code to take
locks has been called, and store a pointer to the PartitionDispatch
object in the AppendState for future use.

I have moved the call to ExecInitAppend. This still uses the previous
locking method, I will work on it in the next version of the patch.

- I am surprised that set_append_subplan_indexes() needs to worry
about multi-level partitioning directly. I would have thought that
Amit's patch would take care of that, just returning a Bitmapset of
indexes which this function could use directly. It also doesn't seem
like a very good idea to convert the Bitmapset (subplans) into a list
of integers (node->subplan_indexes), as set_append_subplan_indexes()
does at the bottom. The Bitmapset will be a lot more efficient; we
should be able to just iterate over that directly rather than
converting it into a List. Note that a Bitmapset can be created with
a single palloc, but an integer list needs one per list element plus
one for the list itself.

The function get_partitions_from_clauses returns the Bitmap set of
partitions for a level of partition. So when the BitmapSet that
indicates a child partitioned table, set_append_subplan_indexes loops
throgh again till it gets the list of all leaf indexes.

I am working on the other comments and will post the patch along with
rebasing to v14 of Amit's patch soon.

--

PFA the updated patch, this can be applied over the v13 patches [1]/messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp
over commit 487a0c1518af2f3ae2d05b7fd23d636d687f28f3

[1]: /messages/by-id/df609168-b7fd-4c0b-e9b2-6e398d411e27@lab.ntt.co.jp

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning_v6.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning_v6.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index f07ac15..b4a4965 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,6 +34,7 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -272,11 +273,14 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+									int rt_index, List *clauses,
+									ParamListInfo base_prmlist,
+									ExprContext *exontext);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -286,7 +290,8 @@ static bool partition_cmp_args(PartitionKey key, int partattoff,
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 
@@ -1659,6 +1664,37 @@ get_partition_qual_relid(Oid relid)
 	return result;
 }
 
+/* get_leaf_part_recurse
+ *		Get the leaf oids for the given rel.
+ */
+void
+get_leaf_part_recurse(Relation rel, List **leaf_part_oids)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+	int			i;
+
+	check_stack_depth();
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		Oid			partrelid = partdesc->oids[i];
+
+		if (get_rel_relkind(partrelid) != RELKIND_PARTITIONED_TABLE)
+			*leaf_part_oids = lappend_oid(*leaf_part_oids, partrelid);
+		else
+		{
+			/*
+			 * We assume all tables in the partition tree were already locked
+			 * by the caller.
+			 */
+			Relation	partrel = heap_open(partrelid, NoLock);
+
+			get_leaf_part_recurse(partrel, leaf_part_oids);
+			heap_close(partrel, NoLock);
+		}
+	}
+}
+
 /*
  * get_partitions_from_clauses
  *		Determine the set of partitions of relation that will satisfy all
@@ -1669,7 +1705,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1703,7 +1740,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 	}
 
 	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+												 partclauses, base_prmlist,
+												 econtext);
 
 	return result;
 }
@@ -1719,7 +1757,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
-									List *clauses)
+									List *clauses, ParamListInfo base_prmlist,
+									ExprContext *econtext)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
 	Bitmapset *result = NULL;
@@ -1735,7 +1774,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, base_prmlist,
+											 econtext);
 
 	/*
 	 * The analysis of the matched clauses done by
@@ -1797,7 +1837,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 
 			arg_partset = get_partitions_from_clauses_recurse(relation,
 															  rt_index,
-															  arg_clauses);
+															  arg_clauses,
+															  base_prmlist,
+															  econtext);
 
 			/*
 			 * Partition sets obtained from mutually-disjunctive clauses are
@@ -1853,7 +1895,8 @@ static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -1893,6 +1936,9 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				continue;
 			}
 		}
+		/* when called from ExecReScanAppend */
+		else if (IsA(lfirst(lc), ExprState))
+			clause = ((ExprState *) lfirst(lc))->expr;
 		else
 			clause = (Expr *) lfirst(lc);
 
@@ -2305,7 +2351,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
-										&keys->maxkeys[i]))
+										&keys->maxkeys[i], base_prmlist,
+										econtext))
 			{
 				keys->n_maxkeys++;
 				keys->max_incl = incl;
@@ -2317,12 +2364,14 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->eqkeys[i]))
+											&keys->eqkeys[i], base_prmlist,
+											econtext))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->maxkeys[i]))
+											&keys->maxkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_maxkeys++;
 					keys->max_incl = true;
@@ -2330,7 +2379,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->minkeys[i]))
+											&keys->minkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_minkeys++;
 					keys->min_incl = true;
@@ -2338,7 +2388,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
-											 &keys->minkeys[i]))
+											 &keys->minkeys[i], base_prmlist,
+											 econtext))
 			{
 				keys->n_minkeys++;
 				keys->min_incl = incl;
@@ -2426,7 +2477,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2467,6 +2519,46 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (base_prmlist)
+					{
+						Node	   *n;
+
+						n = eval_const_expressions_from_list(base_prmlist,
+															 (Node *) expr);
+
+						if (IsA(n, Const))
+						{
+							*value = ((Const *) n)->constvalue;
+							return true;
+						}
+					}
+					return false;
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
+			return false;
+
 		default:
 			return false;
 	}
@@ -2728,10 +2820,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, &leftarg_const,
+								 NULL, NULL))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, &rightarg_const,
+								 NULL, NULL))
 		return false;
 
 	/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 447f69d..06c62cb 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1347,6 +1347,21 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	if (es->format == EXPLAIN_FORMAT_TEXT)
 		appendStringInfoChar(es->str, '\n');
 
+	/* run-time pruning information for Append node */
+	if (es->analyze && IsA(plan, Append))
+	{
+		Append	   *append = (Append *) plan;
+
+		if (append->base_params)
+			show_scan_qual(append->base_params, "Runtime Partition Pruning",
+						   planstate, ancestors, es);
+
+		if (append->join_clauses)
+			show_scan_qual(append->join_clauses,
+						   "Runtime Partition Pruning Join Filter",
+						   planstate, ancestors, es);
+	}
+
 	/* target list */
 	if (es->verbose)
 		show_plan_tlist(planstate, ancestors, es);
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index d275cef..187d30f 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -23,8 +23,6 @@
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
-static PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
-								 int *num_parted, List **leaf_part_oids);
 static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -275,7 +273,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
  * All the relations in the partition tree (including 'rel') must have been
  * locked (using at least the AccessShareLock) by the caller.
  */
-static PartitionDispatch *
+PartitionDispatch *
 RelationGetPartitionDispatchInfo(Relation rel,
 								 int *num_parted, List **leaf_part_oids)
 {
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 1d2fb35..9d36657 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,7 +57,9 @@
 
 #include "postgres.h"
 
+#include "catalog/pg_inherits_fn.h"
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -108,6 +110,65 @@ exec_append_initialize_next(AppendState *appendstate)
 }
 
 /* ----------------------------------------------------------------
+ *		set_append_subplan_indexes
+ *
+ *		Determine the subplans to scan based on the clauses.
+ * ----------------------------------------------------------------
+ */
+static void
+set_append_subplan_indexes(AppendState *node, List *clauses)
+{
+	Bitmapset  *partset = NULL,
+			   *subplans = NULL;
+	PartitionDispatch *pd = (PartitionDispatch *) node->pd,
+				parent,
+			   *p1;
+	List	   *parents = NIL;
+	int			cur_index,
+				i;
+
+	parents = lappend(parents, &pd[0]);
+	node->subplan_indexes = NIL;
+	do
+	{
+		p1 = linitial(parents);
+		parent = *p1;
+
+		partset = get_partitions_from_clauses(parent->reldesc,
+											  1,
+											  list_copy(clauses),
+											  node->es_param_list_info,
+											  node->ps.ps_ExprContext);
+
+		if (!bms_is_empty(partset))
+		{
+			while ((cur_index = bms_first_member(partset)) >= 0)
+			{
+				if (parent->indexes[cur_index] >= 0)
+					subplans =
+						bms_add_member(subplans, parent->indexes[cur_index]);
+				else
+					parents =
+						lappend(parents, &pd[-parent->indexes[cur_index]]);
+			}
+		}
+		parents = list_delete_first(parents);
+	} while (parents);
+
+	if (!bms_is_empty(subplans))
+	{
+		while ((i = bms_first_member(subplans)) >= 0)
+		{
+			int			index = node->append_paths_array[i];
+
+			if (index >= 0)
+				node->subplan_indexes = lappend_int(node->subplan_indexes,
+													index);
+		}
+	}
+}
+
+/* ----------------------------------------------------------------
  *		ExecInitAppend
  *
  *		Begin all of the subscans of the append node.
@@ -151,17 +212,22 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->subplan_indexes = NIL;
+	appendstate->as_whichpartition = -1;
+	appendstate->append_paths_size = node->append_paths_size;
+	appendstate->append_paths_array = node->append_paths_array;
+	appendstate->parentoid = node->parentoid;
 
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
 
+	/* create expression context for node */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * append nodes have Result slots, which hold pointers to tuples, so we
+	 * have to initialize them.
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
@@ -178,9 +244,39 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		i++;
 	}
 
-	/*
-	 * initialize output tuple type
-	 */
+	if (node->base_params)
+	{
+		appendstate->base_params = node->base_params;
+		appendstate->es_param_list_info = estate->es_param_list_info;
+	}
+
+	if (node->join_clauses)
+		appendstate->join_clauses = ExecInitExprList(node->join_clauses,
+													(PlanState *) appendstate);
+
+	if(node->base_params || node->join_clauses)
+	{
+		Relation	rel;
+		PartitionDispatch *pd;
+		List	   *leaf_part_oids = NIL;
+		int num_parted;
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		rel = relation_open(node->parentoid, NoLock);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		relation_close(rel, NoLock);
+		appendstate->pd = pd;
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+	}
+
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
@@ -204,6 +300,19 @@ ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
 
+	/* Determine the subplans to scan depending on the base_params */
+	if (node->base_params && node->as_whichpartition == -1)
+	{
+		set_append_subplan_indexes(node, node->base_params);
+		if (node->subplan_indexes)
+		{
+			node->as_whichplan = linitial_int(node->subplan_indexes);
+			node->as_whichpartition = 0;
+		}
+		else
+			node->as_whichplan = 0;
+	}
+
 	for (;;)
 	{
 		PlanState  *subnode;
@@ -212,8 +321,19 @@ ExecAppend(PlanState *pstate)
 		CHECK_FOR_INTERRUPTS();
 
 		/*
+		 * end scan if no subplan is selected for the current join_clauses
+		 * and/or base_params
+		 */
+		if ((node->join_clauses || node->base_params)
+			&& node->as_whichpartition == -1)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/*
 		 * figure out which subplan we are currently processing
 		 */
+		if (node->as_whichpartition != -1)
+			node->as_whichplan = list_nth_int(node->subplan_indexes,
+											  node->as_whichpartition);
 		subnode = node->appendplans[node->as_whichplan];
 
 		/*
@@ -237,7 +357,22 @@ ExecAppend(PlanState *pstate)
 		 * ExecInitAppend.
 		 */
 		if (ScanDirectionIsForward(node->ps.state->es_direction))
-			node->as_whichplan++;
+		{
+			/*
+			 * For runtime partition pruning, goto the next valid partition
+			 * index
+			 */
+			if (node->subplan_indexes)
+			{
+				if (++node->as_whichpartition < list_length(node->subplan_indexes))
+					node->as_whichplan = list_nth_int(node->subplan_indexes,
+													  node->as_whichpartition);
+				else
+					return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+			}
+			else
+				node->as_whichplan++;
+		}
 		else
 			node->as_whichplan--;
 		if (!exec_append_initialize_next(node))
@@ -280,6 +415,10 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/* Determine subplans to scan based on the new Params */
+	if (node->ps.chgParam != NULL && node->join_clauses)
+		set_append_subplan_indexes(node, node->join_clauses);
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -298,6 +437,16 @@ ExecReScanAppend(AppendState *node)
 		if (subnode->chgParam == NULL)
 			ExecReScan(subnode);
 	}
-	node->as_whichplan = 0;
+
+	if (node->subplan_indexes)
+	{
+		node->as_whichplan = linitial_int(node->subplan_indexes);
+		node->as_whichpartition = 0;
+	}
+	else
+	{
+		node->as_whichplan = 0;
+		node->as_whichpartition = -1;
+	}
 	exec_append_initialize_next(node);
 }
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d9ff8a7..4af11e8 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,11 @@ _copyAppend(const Append *from)
 	 */
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
+	COPY_NODE_FIELD(base_params);
+	COPY_NODE_FIELD(join_clauses);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_POINTER_FIELD(append_paths_array, from->append_paths_size * sizeof(int));
+	COPY_SCALAR_FIELD(append_paths_size);
 
 	return newnode;
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6b087ec..78d064a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -26,6 +26,7 @@
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_proc.h"
 #include "catalog/pg_type.h"
+#include "catalog/pg_inherits_fn.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -144,9 +145,10 @@ static List *get_append_rel_partitions(PlannerInfo *root,
 static List *match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse);
-
+static int	list_member_oid_index(List *list, Oid datum);
 
 /*
  * make_one_rel
@@ -289,6 +291,27 @@ set_base_rel_sizes(PlannerInfo *root)
 		if (root->glob->parallelModeOK)
 			set_rel_consider_parallel(root, rel, rte);
 
+		if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			List	   *leaf_parts = NIL;
+			Relation	parent = relation_open(rte->relid, NoLock);
+			int			i;
+
+			/* fetch the leaf oids of the parent rel */
+			(void) find_all_inheritors(RelationGetRelid(parent), AccessShareLock, NULL);
+			get_leaf_part_recurse(parent, &leaf_parts);
+			relation_close(parent, NoLock);
+
+			/* initialize necessary root variables */
+			root->append_paths_size = list_length(leaf_parts);
+			root->leaf_node_oids = leaf_parts;
+			root->append_paths_array = palloc0(root->append_paths_size * sizeof(int));
+			root->baserestrictinfo_param_indexes = NIL;
+			root->append_paths_count = 0;
+
+			for (i = 0; i < root->append_paths_size; i++)
+				root->append_paths_array[i] = -1;
+		}
 		set_rel_size(root, rel, rti, rte);
 	}
 }
@@ -350,6 +373,40 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	{
 		/* It's an "append relation", process accordingly */
 		set_append_rel_size(root, rel, rti, rte);
+
+		/*
+		 * If this rel is part of a join then collect the necessary join
+		 * clauses required for runtime partition pruning.
+		 */
+		if (rel->joininfo && rel->part_scheme)
+		{
+			List	   *partclauses;
+			bool		contains_param,
+						contains_const,
+						constfalse;
+
+			/*
+			 * Get the clauses that match the partition key
+			 */
+			partclauses = match_clauses_to_partkey(root, rel,
+												   list_copy(rel->joininfo),
+												   &contains_param,
+												   &contains_const,
+												   &constfalse);
+
+			if (partclauses != NIL)
+			{
+				ListCell   *lc;
+
+				foreach(lc, partclauses)
+				{
+					Node	   *n = lfirst(lc);
+
+					if (!list_member(root->join_clauses, n))
+						root->join_clauses = lappend(root->join_clauses, n);
+				}
+			}
+		}
 	}
 	else
 	{
@@ -871,7 +928,8 @@ get_append_rel_partitions(PlannerInfo *root,
 {
 	List   *partclauses;
 	bool	contains_const,
-			constfalse;
+			constfalse,
+			contains_param;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -880,6 +938,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &contains_param,
 										   &contains_const,
 										   &constfalse);
 
@@ -897,7 +956,7 @@ get_append_rel_partitions(PlannerInfo *root,
 		 */
 		if (partclauses != NIL && contains_const)
 			partindexes = get_partitions_from_clauses(parent, rel->relid,
-													  partclauses);
+													  partclauses, NULL, NULL);
 
 		/*
 		 * Else there are no clauses that are useful to prune any paritions,
@@ -964,17 +1023,21 @@ get_append_rel_partitions(PlannerInfo *root,
  *
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
+ *
+ * If the list has a param, *contains_param is set
  */
 static List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse)
 {
 	PartitionScheme	partscheme = rel->part_scheme;
 	List	   *result = NIL;
 	ListCell   *lc;
+	int			clause_index = -1;
 
 	*contains_const = false;
 	*constfalse = false;
@@ -987,6 +1050,8 @@ match_clauses_to_partkey(PlannerInfo *root,
 		Expr   *clause;
 		int		i;
 
+		clause_index++;
+
 		if (IsA(member, RestrictInfo))
 		{
 			RestrictInfo *rinfo = (RestrictInfo *) member;
@@ -1024,14 +1089,24 @@ match_clauses_to_partkey(PlannerInfo *root,
 				bool	arg_matches_key = false,
 						matched_arg_contains_const = false,
 						all_args_constfalse = true;
+				List	   *base_param_indexes;
+
+				/*
+				 * Make a copy of root's baserestrictinfo_param_indexes before
+				 * the recursive call
+				 */
+				base_param_indexes =
+					list_copy(root->baserestrictinfo_param_indexes);
 
 				foreach (lc1, orclause->args)
 				{
 					Node   *arg = lfirst(lc1);
-					bool	contains_const1,
-							constfalse1;
+					bool	contains_const1;
+					bool	constfalse1;
+					bool	contains_param1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 &contains_param1,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1042,8 +1117,22 @@ match_clauses_to_partkey(PlannerInfo *root,
 					/* We got at least one arg that is not constant false. */
 					if (!constfalse1)
 						all_args_constfalse = false;
+
+					/*
+					 * We got at least one arg which is Param, so add the
+					 * current clause_index to the base_param_indexes list.
+					 */
+					if (contains_param1)
+					{
+						*contains_param = true;
+						base_param_indexes =
+							list_append_unique_int(base_param_indexes,
+												   clause_index);
+					}
 				}
 
+				root->baserestrictinfo_param_indexes = base_param_indexes;
+
 				if (arg_matches_key)
 				{
 					result = lappend(result, clause);
@@ -1150,6 +1239,15 @@ match_clauses_to_partkey(PlannerInfo *root,
 				if (contain_volatile_functions((Node *) constexpr))
 					continue;
 
+				if (IsA(constexpr, Param))
+				{
+					*contains_param = true;
+
+					root->baserestrictinfo_param_indexes =
+						list_append_unique_int(root->baserestrictinfo_param_indexes,
+											   clause_index);
+				}
+
 				/*
 				 * Everything seems to be fine, so add it to the list of
 				 * clauses we will use for pruning.
@@ -1249,6 +1347,26 @@ match_clauses_to_partkey(PlannerInfo *root,
 	return result;
 }
 
+/* list_member_oid_index
+ *	  Returns the index of the given datum in the oid list.
+ *
+ *	It returns -1 if the datum is not found.
+ */
+static int
+list_member_oid_index(List *list, Oid datum)
+{
+	int			i = 0;
+	const ListCell *cell;
+
+	foreach(cell, list)
+	{
+		if (lfirst_oid(cell) == datum)
+			return i;
+		i++;
+	}
+	return -1;
+}
+
 /*
  * set_append_rel_size
  *	  Set size estimates for a simple "append relation"
@@ -1641,7 +1759,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, rel_appinfos)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
-		int			childRTindex;
+		int			childRTindex,
+					index;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 
@@ -1674,6 +1793,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* only consider non dummy children */
+		index = list_member_oid_index(root->leaf_node_oids, childRTE->relid);
+		if (index >= 0)
+			root->append_paths_array[index] = root->append_paths_count++;
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
@@ -1855,7 +1979,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NULL, 0,
 												  partitioned_rels));
 
 	/*
@@ -1882,7 +2006,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, partial_subpaths, NULL,
 										parallel_workers, partitioned_rels);
 		add_partial_path(rel, (Path *) appendpath);
 	}
@@ -1936,7 +2060,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, required_outer, 0,
+					 create_append_path(root, rel, subpaths, required_outer, 0,
 										partitioned_rels));
 	}
 }
@@ -2173,7 +2297,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index b491fb9..61fe647 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NULL, 0, NIL));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d445477..a47373f 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1001,6 +1001,24 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static List *
+replace_partition_nestloop_params(PlannerInfo *root, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Node	   *n = lfirst(lc);
+
+		new_list = lappend(new_list, replace_nestloop_params(root, n));
+	}
+	return new_list;
+}
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1063,6 +1081,12 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	plan->base_params = best_path->base_params;
+	plan->join_clauses = replace_partition_nestloop_params(root, best_path->join_clauses);
+	plan->parentoid = best_path->parentoid;
+	plan->append_paths_size = best_path->append_paths_size;
+	plan->append_paths_array = best_path->append_paths_array;
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 28093ac..ffefcd0 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NULL,
 								   0,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f620243..12d0f85 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NULL, 0, NIL);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 72f1fa3..fbbcae8 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 68dee0f..12baead 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1208,7 +1208,7 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
+create_append_path(PlannerInfo *root, RelOptInfo *rel, List *subpaths, Relids required_outer,
 				   int parallel_workers, List *partitioned_rels)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
@@ -1253,6 +1253,56 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
 		Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
 	}
 
+	/* Do necessary evaluation needed for runtime partition pruning. */
+	if (root && (required_outer || (rel->baserestrictinfo)))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		pathnode->append_paths_array = root->append_paths_array;
+		pathnode->append_paths_size = root->append_paths_size;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				if (root->join_clauses)
+					pathnode->join_clauses = root->join_clauses;
+
+				if (root->baserestrictinfo_param_indexes && rel->baserestrictinfo)
+				{
+					ListCell   *lc;
+					List	   *quals = list_copy(rel->baserestrictinfo);
+
+					pathnode->base_params = NIL;
+
+					/*
+					 * collect the clauses from baserestrictinfo for partition
+					 * pruning
+					 */
+					foreach(lc, root->baserestrictinfo_param_indexes)
+					{
+						int			index = lfirst_int(lc);
+						Node	   *member = list_nth(quals, index);
+						Expr	   *clause;
+
+						if (IsA(member, RestrictInfo))
+							clause = ((RestrictInfo *) member)->clause;
+						else
+							clause = (Expr *) member;
+
+						pathnode->base_params = lappend(pathnode->base_params,
+														clause);
+					}
+				}
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index a968fa4..545a34b 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1555,6 +1555,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1566,11 +1569,27 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+	/*
+	 * To determine whether the appendrel is applicable for runtime pruning or
+	 * not, we fetch the clause from the join clause.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 2041de5..5a8acd9 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,7 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses);
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext);
+extern void get_leaf_part_recurse(Relation rel, List **leaf_part_oids);
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 64e5aab..6e7a841 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -49,6 +49,9 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
+								 int *num_parted, List **leaf_part_oids);
+
 extern void ExecSetupPartitionTupleRouting(Relation rel,
 							   Index resultRTindex,
 							   EState *estate,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e05bc04..0c3be05 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -41,6 +41,7 @@
 struct ExprState;				/* forward references in this file */
 struct ExprContext;
 struct ExprEvalStep;			/* avoid including execExpr.h everywhere */
+struct PartitionDispatchData;
 
 typedef Datum (*ExprStateEvalFunc) (struct ExprState *expression,
 									struct ExprContext *econtext,
@@ -1006,6 +1007,17 @@ typedef struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+
+	/* for runtime partition pruning */
+	Oid			parentoid;		/* oid of the parent */
+	int		   *append_paths_array;
+	int			append_paths_size;	/* size of append_paths_array */
+	List	   *base_params;	/* base restrictinfo on partition keys */
+	List	   *join_clauses;	/* join clauses on partiton keys */
+	ParamListInfo es_param_list_info;
+	List	   *subplan_indexes;	/* List of subplan indexes to scan */
+	int			as_whichpartition;	/* current index to scan in index */
+	struct PartitionDispatchData **pd;
 } AppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 9b38d44..698322d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -248,6 +249,13 @@ typedef struct Append
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *appendplans;
+
+	/* for runtime partition pruning */
+	Oid			parentoid;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	List	   *base_params;
+	List	   *join_clauses;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 94c2e8d..611fb74 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -317,6 +317,17 @@ typedef struct PlannerInfo
 
 	/* optional private data for join_search_hook, e.g., GEQO */
 	void	   *join_search_private;
+
+	/* These fields accumulate data required for runtime partition pruning. */
+	int		   *append_paths_array; /* array for append subplans info */
+	int			append_paths_size;	/* size of append_paths_array */
+	int			append_paths_count; /* current count of append_paths_array */
+	List	   *leaf_node_oids; /* leaf oids of current rel */
+	List	   *baserestrictinfo_param_indexes; /* index from baserestrictinfo
+												 * list with partition related
+												 * clauses */
+	List	   *join_clauses;	/* join clause which are required for
+								 * partition pruning */
 } PlannerInfo;
 
 
@@ -1289,6 +1300,13 @@ typedef struct AppendPath
 	/* RT indexes of non-leaf tables in a partition tree */
 	List	   *partitioned_rels;
 	List	   *subpaths;		/* list of component Paths */
+
+	/* for runtime partition pruning */
+	Oid			parentoid;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	List	   *base_params;
+	List	   *join_clauses;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..f8e7660 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index c1f2fc9..7c1fbfd 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -63,8 +63,8 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
-				   Relids required_outer, int parallel_workers,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
+				   List *subpaths, Relids required_outer, int parallel_workers,
 				   List *partitioned_rels);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
#35Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#30)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Fri, Dec 8, 2017 at 3:37 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 7 December 2017 at 23:56, Beena Emerson <memissemerson@gmail.com> wrote:

Currently Amit's v13 patches do not apply on the HEAD and I was
working on 487a0c1518af2f3ae2d05b7fd23d636d687f28f3 which is the last
commit where all Amit's v13 patches applies cleanly.

Thanks.

I was just looking over this and was wondering about the following case:

drop table if exists p;
create table p (a int not null, b int not null) partition by range (a);
create table p1 partition of p for values from (0) to (1000);
create table p2 partition of p for values from (1000) to (2000);
create table p3 partition of p for values from (2000) to (3000);
create table p4 partition of p for values from (3000) to (4000);

create index on p1 (a);
create index on p2 (a);
create index on p3 (a);
create index on p4 (a);

insert into p select x,x from generate_series(1,3999) x;

drop table if exists t;
create table t (a int not null);

insert into t select generate_Series(1,10);

analyze p;

analyze t;

set enable_mergejoin=0;
set enable_hashjoin=0;

explain analyze select * from p inner join t on p.a = t.a;

The patch gives me:

QUERY PLAN
----------------------------------------------------------------------------------------
Nested Loop (actual time=0.032..0.159 rows=10 loops=1)
-> Seq Scan on t (actual time=0.012..0.013 rows=10 loops=1)
-> Append (actual time=0.004..0.013 rows=1 loops=10)
-> Index Scan using p1_a_idx on p1 (actual time=0.004..0.004
rows=1 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p2_a_idx on p2 (actual time=0.003..0.003
rows=0 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p3_a_idx on p3 (actual time=0.002..0.002
rows=0 loops=10)
Index Cond: (a = t.a)
-> Index Scan using p4_a_idx on p4 (actual time=0.003..0.003
rows=0 loops=10)
Index Cond: (a = t.a)
Planning time: 0.472 ms
Execution time: 0.241 ms
(13 rows)

but I expected to get (never executed) for p2, p3 and p4.

The following code makes me think you intend this to work:

@@ -280,6 +438,10 @@ ExecReScanAppend(AppendState *node)
{
int i;

+ /* Determine subplans to scan based on the new Params */
+ if (node->ps.chgParam != NULL && node->join_clauses)
+ set_append_subplan_indexes(node, node->join_clauses);
+

It just does not due to the node->join_clauses being NULL.

Thank you for your tests. I am working on this.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#36Beena Emerson
memissemerson@gmail.com
In reply to: Jesper Pedersen (#32)
Re: [HACKERS] Runtime Partition Pruning

Hello Jesper,

On Tue, Dec 12, 2017 at 4:04 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Hi Beena,

On 12/07/2017 02:22 AM, Beena Emerson wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

As the current patch conflicts with [1] could you provide a rebased version
?

Thanks in advance !

[1]
/messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp

I am aware of this and will post a rebased version soon.

Thank you,

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#37David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#34)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 13 December 2017 at 00:33, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch, this can be applied over the v13 patches [1]
over commit 487a0c1518af2f3ae2d05b7fd23d636d687f28f3

Hi Beena,

Thanks for posting an updated patch.

I've been looking over this and I think that the use of the
PartitionDispatch in set_append_subplan_indexes is not correct. What
we need here is the index of the Append's subnode and that's not what
RelationGetPartitionDispatchInfo() gives you. Remember that some
partitions could have been pruned away already during planning.

This quick example shows that the partition selection is not correct.

create table p (a int, b int) partition by range (a);

create table p_a_neg partition of p for values from (minvalue) to (0)
partition by range (b);
create table p_a_pos partition of p for values from (0) to (maxvalue)
partition by range (b);

create table p_a_neg_b_neg partition of p_a_neg for values from
(minvalue) to (0);
create table p_a_neg_b_pos partition of p_a_neg for values from (0) to
(maxvalue);

create table p_a_pos_b_neg partition of p_a_pos for values from
(minvalue) to (0);
create table p_a_pos_b_pos partition of p_a_pos for values from (0) to
(maxvalue);

prepare q1 (int, int) as select * from p where a = $1 and b = $1;

explain analyze execute q1 (-1,-1); -- this works.
QUERY PLAN
--------------------------------------------------------------------------------------------------------------
Append (cost=0.00..175.60 rows=4 width=8) (actual time=1.099..1.099
rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_neg (cost=0.00..43.90 rows=1 width=8)
(actual time=0.023..0.023 rows=0 loops=1)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_neg (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
(12 rows)

explain analyze execute q1 (-1,1); -- should scan p_a_neg_b_pos, but does not.
QUERY PLAN
--------------------------------------------------------------------------------------------------------------
Append (cost=0.00..175.60 rows=4 width=8) (actual
time=758996.359..758996.359 rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_neg (cost=0.00..43.90 rows=1 width=8)
(actual time=0.056..0.056 rows=0 loops=1)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_neg (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
(12 rows)

So, I started to look at what the best way to put this right might be.
I see that since Parallel Append was committed that the subnodes are
now sorted in cost order inside create_append_path(), so likely we'll
want to delay figuring out the subpath list indexes until after that's
done since sorting would scramble our index arrays. We could simply
look at the subpaths at the end of create_append_path() and create
some sort of new matrix type that can accept the output of Amit's
get_partitions_from_clauses() and translate that Bitmapset into the
subpath indexes (another Bitmapset). This will also need to work for
sub-partitions too, so this matrix must be some sort of tree that we
can descend into when we see that get_partitions_from_clauses returned
a bit for a sub-partition instead of a leaf-partition.

I bashed this idea around a bit and I came up with the attached. It's
very far from complete and in a very WIP state. I've not really done
anything to make the correct clause list available in nodeAppend.c
yet, but I think the code that's there is worthy of a look. I've not
done that much work on the new choose_next_subplan* functions in
nodeAppend.c. I just modified choose_next_subplan_locally to show how
this set of functions need to take into account the subnode bitmap set
of valid partitions to scan. Perhaps some special case is needed to
have these functions ignore the Bitmapset when runtime pruning is
disabled (perhaps a completely new set of the functions is needed to
support the selection of the next non-pruned partition). Although,
probably that can be debated a bit later as it's a fairly minor detail
for now.

My patch also lacks any means to extract the Params during
match_clauses_to_partkey(), or at least most of the cases. I've just
added 1 case there. I did this because I thought it was better to
extract the ParamIds rather than a bool to say we've matched params.
This way we can only reevaluate which subplans to look at on rescan of
an Append if and only if the params we actually care about have
changed. I've not given this part a huge amount of thought yet.

I'm a little unsure where to go from here. Obviously, this is quite a
major rewrite of your patch. The parts that I've got missing likely
can use quite a bit of the stuff you've already written, but that
needs some review. I wanted to post this now as I know you're busy
working on this to rebase it on parallel Append and to also address
Robert's concerns, which all seem valid to me.

What's the best way for us to coordinate our efforts on this? Maybe
you could look at my patch and sanity check it to ensure I'm not
taking anything in the wrong direction?

I also think that MergeAppend needs similar work, but I think it's
best to get the Append case working first, or perhaps that's another
patch...

Please find attached my patch, which is based directly on top of
Amit's faster partition pruning v14 [1]/messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp, which I patched against
4034db215b9

[1]: /messages/by-id/9b98fc47-34b8-0ab6-27fc-c8a0889f2e5b@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v1.patchapplication/octet-stream; name=runtime_prune_drowley_v1.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 7e3a777..d58c50b 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -39,6 +39,7 @@
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -1712,6 +1713,122 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		subpath indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentid = rel->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that's already been pruned. Set them all to
+	 * this initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths who's rel is not directly parented by rel.  We'll
+		 * process any we skip here later when looping through partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subnode.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the tables returned by get_partitions_from_clauses may be other
+	 * partitioned tables.  Unlike the case above, these won't be subnodes of
+	 * the Append.  To handle these we must make use of another
+	 * PartitionPruneInfo which we must use to recursively search until we
+	 * reach the leaf partitions.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the relation being queried.  We
+		 * only care about sub-partition parents here.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel. We'll search until we get down to the leaf partitions by
+		 * recursively calling ourself.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Some of the bits returned by get_partitions_from_clauses will
+		 * reference a partition parent.  Here we mark that the index
+		 * references a partition parent, this allows a partition search
+		 * to recursively search for all subnodes matching a search.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..fc66124 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,6 +57,7 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
@@ -82,6 +83,10 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node,
+						   EState *estate, PartitionPruneInfo *pinfo,
+						   List *clauses);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -129,6 +134,18 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->as_nplans = nplans;
 
 	/*
+	 * The presence  of a part_prune_info means that run-time pruning is
+	 * enabled, so here we'll determine which subplans need to be scanned.
+	 */
+	if (node->part_prune_info)
+		set_valid_runtime_subplans(appendstate, estate, node->part_prune_info,
+								   NULL /* XXX get the clause list */);
+
+	/* else, mark all subplans as requiring a scan */
+	else
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+	/*
 	 * Miscellaneous initialization
 	 *
 	 * Append plans don't have expression contexts because they never call
@@ -366,22 +383,19 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
 	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
 	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -392,6 +406,7 @@ choose_next_subplan_locally(AppendState *node)
  *      Try to pick a plan which doesn't commit us to doing much
  *      work locally, so that as much work as possible is done in
  *      the workers.  Cheapest subplans are at the end.
+ *      XXX needs updated to use node->as_valid_subplans
  * ----------------------------------------------------------------
  */
 static bool
@@ -450,6 +465,7 @@ choose_next_subplan_for_leader(AppendState *node)
  *		in order of descending cost and then spreads out the
  *		workers as evenly as possible across the remaining partial
  *		plans.
+ *		XXX needs updated to use node->as_valid_subplans
  * ----------------------------------------------------------------
  */
 static bool
@@ -526,3 +542,45 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * get_valid_runtime_subplans
+ *		Recursively descend through each PartitionPruneInfo, along the way
+ *		determine which subnode indexes we need to scan to get all rows that
+ *		satisfy 'clauses'.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node, EState *estate,
+						   PartitionPruneInfo *pinfo, List *clauses)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	Oid				relid;
+	int				i;
+
+	relid = getrelid(pinfo->parentid, estate->es_range_table);
+
+	rel = relation_open(relid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, pinfo->parentid, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			node->as_valid_subplans = bms_add_member(node->as_valid_subplans,
+													 pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans(node, estate, pinfo->subpartindex[i],
+									   clauses);
+		else
+		{
+			/* If this happens then we're somehow missing an Append subnode */
+			elog(ERROR, "Partition missing from Append subnodes");
+		}
+	}
+
+	relation_close(rel, NoLock);
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index ae30072..7c7f985 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -84,6 +84,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1064,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * 1-bit in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* switch off bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d222eff..156022a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -144,6 +144,7 @@ static List *get_append_rel_partitions(PlannerInfo *root,
 static List *match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse);
 
@@ -877,6 +878,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +887,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -967,6 +970,7 @@ static List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,17 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+				{
+					/*
+					 * XXX there's a bunch of other places to gather Params
+					 * from. I've not done that yet.
+					 */
+					Param *param = (Param *) constexpr;
+					*paramids = bms_add_member(*paramids, param->paramid);
+				}
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1715,6 +1729,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	List	   *partitioned_rels = NIL;
 	RangeTblEntry *rte;
 	double		partial_rows = -1;
+	bool		runtimeprune = false;
 
 	/*
 	 * AppendPath generated for partitioned tables must record the RT indexes
@@ -1739,7 +1754,17 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		rte = planner_rt_fetch(rel->relid, root);
 		if (rte->rtekind == RTE_RELATION &&
 			rte->relkind == RELKIND_PARTITIONED_TABLE)
-		partitioned_rels = rel->live_partitioned_rels;
+		{
+			partitioned_rels = rel->live_partitioned_rels;
+
+			/*
+			 * XXX Should only enable if we find Params matching the partkey.
+			 * We may have to record the ParamIds we matched to the partition
+			 * key in RelOptInfo, but I'm hoping there is a better way.
+			 */
+			if (rel->reloptkind == RELOPT_BASEREL)
+				runtimeprune = true;
+		}
 	}
 	else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
 	{
@@ -1911,9 +1936,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
-												  partitioned_rels, -1));
+												  partitioned_rels, -1,
+												  runtimeprune));
 
 	/*
 	 * Consider an append of unordered, unparameterized partial paths.  Make
@@ -1953,10 +1979,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
-										partitioned_rels, -1);
+										partitioned_rels, -1,
+										runtimeprune);
 
 		/*
 		 * Make sure any subsequent partial paths use the same row count
@@ -2002,10 +2029,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
-										partitioned_rels, partial_rows);
+										partitioned_rels, partial_rows,
+										runtimeprune);
 		add_partial_path(rel, (Path *) appendpath);
 	}
 
@@ -2058,9 +2086,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
-										partitioned_rels, -1));
+										partitioned_rels, -1,
+										runtimeprune));
 	}
 }
 
@@ -2323,8 +2352,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..15d1426 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,8 +1232,8 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index f6c83d0..b72c30e 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,8 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1061,7 +1062,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   best_path->partprune);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5297,7 +5299,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5309,7 +5312,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 7a09f07..2fe1d4a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,14 +3678,16 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
 								   0,
 								   false,
 								   NIL,
-								   -1);
+								   -1,
+								   false);
 			path->pathtarget = target;
 		}
 		else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index a24e8ac..cc08769 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,8 +590,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
 
@@ -702,8 +702,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 54126fb..8832b9b 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,11 +1211,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows)
+				   List *partitioned_rels, double rows,
+				   bool runtimeprune)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
 	ListCell   *l;
@@ -1269,6 +1272,11 @@ create_append_path(RelOptInfo *rel,
 	if (rows >= 0)
 		pathnode->path.rows = rows;
 
+	/* Generate data structure required for run-time partition pruning */
+	if (runtimeprune)
+		pathnode->partprune = make_partition_pruneinfo(root, rel,
+													   partitioned_rels,
+													   subpaths);
 	return pathnode;
 }
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..9decfc5 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -74,4 +75,8 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
+
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 3b62a97..25910c4 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int nextbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1a35c5c..a04e6b6 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -999,6 +999,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1013,6 +1014,7 @@ struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Bitmapset  *as_valid_subplans;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
 	bool		(*choose_next_subplan) (AppendState *);
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c5b5115..cbd76ca 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -218,6 +218,7 @@ typedef enum NodeTag
 	T_IndexOptInfo,
 	T_ForeignKeyOptInfo,
 	T_ParamPathInfo,
+	T_PartitionPruneInfo,
 	T_Path,
 	T_IndexPath,
 	T_BitmapHeapPath,
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 02fb366..64f3d75 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Details of how to prune surplus subplans or NULL when disabled enabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 074ae0a..d056655 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,20 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*
+ * PartitionPruneInfo
+ *
+ * Used for execution time partition pruning of Append subnodes
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Index		parentid; /* relid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index bcb669d..02738a3 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1295,6 +1295,11 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	/*
+	 * Details of how to prune surplus subplans or NULL when disabled enabled.
+	 */
+	PartitionPruneInfo *partprune;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 00c134d..892469c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,11 +64,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows);
+				   List *partitioned_rels, double rows, bool runtimeprune);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
#38Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#37)
Re: [HACKERS] Runtime Partition Pruning

Hi.

On 2017/12/16 15:05, David Rowley wrote:

On 13 December 2017 at 00:33, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch, this can be applied over the v13 patches [1]
over commit 487a0c1518af2f3ae2d05b7fd23d636d687f28f3

Hi Beena,

Thanks for posting an updated patch.

I've been looking over this and I think that the use of the
PartitionDispatch in set_append_subplan_indexes is not correct. What
we need here is the index of the Append's subnode and that's not what
RelationGetPartitionDispatchInfo() gives you. Remember that some
partitions could have been pruned away already during planning.

A somewhat similar concern is being discussed on the "UPDATE partition
key" thread [1]/messages/by-id/c5e1d4ad-d243-52c5-608b-5dbb7183e465@lab.ntt.co.jp. In that case, ExecInitModifyTable(), when initializing
tuple routing information to handle the "update partition key" case, will
have to deal with the fact that there might be fewer sub-plans in the
ModifyTable node than there are partitions in the partition tree. That
is, source partitions that planner would have determined after pruning,
could be fewer than possible target partitions for rows from the source
partitions to move to, of which the latter consists of *all* partitions.
So, we have to have a mapping from leaf partition indexes as figured out
by RelationGetPartitionDispatchInfo() (indexes that are offsets into a
global array for *all* partitions), to sub-plan indexes which are offsets
into the array for only those partitions that have a sub-plan. Such
mapping is built (per the latest patch on that thread) by
ExecSetupPartitionTupleRouting() in execPartition.c.

We could do something similar here using a similar code structure. Maybe,
add a ExecSetupPartitionRuntimePruning() in execPartition.c (mimicking
ExecSetupPartitionTupleRouting), that accepts AppendState node.
Furthermore, it might be a good idea to have something similar to
ExecFindPartition(), say, ExecGetPartitions(). That is, we have new
functions for run-time pruning that are counterparts to corresponding
functions for tuple routing.

Thanks,
Amit

[1]: /messages/by-id/c5e1d4ad-d243-52c5-608b-5dbb7183e465@lab.ntt.co.jp
/messages/by-id/c5e1d4ad-d243-52c5-608b-5dbb7183e465@lab.ntt.co.jp

#39Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#31)
Re: [HACKERS] Runtime Partition Pruning

On 2017/12/09 0:57, Robert Haas wrote:

On Thu, Dec 7, 2017 at 2:22 AM, Beena Emerson <memissemerson@gmail.com> wrote:

I have added the partition quals that are used for pruning.

PFA the updated patch. I have changed the names of variables to make
it more appropriate, along with adding more code comments and doing
some refactoring and other code cleanups.

- I am surprised that set_append_subplan_indexes() needs to worry
about multi-level partitioning directly. I would have thought that
Amit's patch would take care of that, just returning a Bitmapset of
indexes which this function could use directly.

Actually, the partition.c code that my patch adds is limited to consider
one partitioned table at a time, not the whole tree. As of 0a480502b09
[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=0a480502b09
in a partition tree. In each such call, we call partition.c to perform
partition pruning for the given partitioned table.

In the run-time pruning case, we should get, via Append, a list of pruning
clauses for each partitioned table in the tree that survived plan-time
pruning. Then, just like ExecFindPartition() calls
get_partition_for_tuple() for each partitioned table until we get to a
leaf partition, we should call partition.c for each un-pruned partitioned
table that have run-time pruning clauses associated.

Thanks,
Amit

[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=0a480502b09

#40David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#38)
Re: [HACKERS] Runtime Partition Pruning

On 18 December 2017 at 21:31, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2017/12/16 15:05, David Rowley wrote:

I've been looking over this and I think that the use of the
PartitionDispatch in set_append_subplan_indexes is not correct. What
we need here is the index of the Append's subnode and that's not what
RelationGetPartitionDispatchInfo() gives you. Remember that some
partitions could have been pruned away already during planning.

A somewhat similar concern is being discussed on the "UPDATE partition
key" thread [1]. In that case, ExecInitModifyTable(), when initializing
tuple routing information to handle the "update partition key" case, will
have to deal with the fact that there might be fewer sub-plans in the
ModifyTable node than there are partitions in the partition tree. That
is, source partitions that planner would have determined after pruning,
could be fewer than possible target partitions for rows from the source
partitions to move to, of which the latter consists of *all* partitions.
So, we have to have a mapping from leaf partition indexes as figured out
by RelationGetPartitionDispatchInfo() (indexes that are offsets into a
global array for *all* partitions), to sub-plan indexes which are offsets
into the array for only those partitions that have a sub-plan. Such
mapping is built (per the latest patch on that thread) by
ExecSetupPartitionTupleRouting() in execPartition.c.

Surely this is a different problem? With UPDATE of a partition key, if
the planner eliminates all but 1 partition the UPDATE could cause that
tuple to be "moved" into any leaf partition, very possibly one that's
been eliminated during planning.

In the case of runtime Append pruning, we can forget about all
partitions that the planner managed to eliminate, we'll never need to
touch those, ever. All we care about here is trying to reduce the
number of partitions down further using values that were not available
during planning.

We could do something similar here using a similar code structure. Maybe,
add a ExecSetupPartitionRuntimePruning() in execPartition.c (mimicking
ExecSetupPartitionTupleRouting), that accepts AppendState node.
Furthermore, it might be a good idea to have something similar to
ExecFindPartition(), say, ExecGetPartitions(). That is, we have new
functions for run-time pruning that are counterparts to corresponding
functions for tuple routing.

Seems to me in this case we're better to build this structure during
planning and save it with the plan so that it can be used over and
over, rather than building it again and again each time the plan is
executed. Likely a common use case for run-time pruning is when the
plan is going to be used multiple times with different parameters, so
we really don't want to repeat any work that we don't have to here.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#41Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#37)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

Thank you for reviewing and looking at this.

I have attached the WIP patch which incorporates some of Robert's
comments and is rebased over Amit's v14 patch.

On Sat, Dec 16, 2017 at 11:35 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 13 December 2017 at 00:33, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch, this can be applied over the v13 patches [1]
over commit 487a0c1518af2f3ae2d05b7fd23d636d687f28f3

Hi Beena,

Thanks for posting an updated patch.

I've been looking over this and I think that the use of the
PartitionDispatch in set_append_subplan_indexes is not correct. What
we need here is the index of the Append's subnode and that's not what
RelationGetPartitionDispatchInfo() gives you. Remember that some
partitions could have been pruned away already during planning.

This quick example shows that the partition selection is not correct.

create table p (a int, b int) partition by range (a);

create table p_a_neg partition of p for values from (minvalue) to (0)
partition by range (b);
create table p_a_pos partition of p for values from (0) to (maxvalue)
partition by range (b);

create table p_a_neg_b_neg partition of p_a_neg for values from
(minvalue) to (0);
create table p_a_neg_b_pos partition of p_a_neg for values from (0) to
(maxvalue);

create table p_a_pos_b_neg partition of p_a_pos for values from
(minvalue) to (0);
create table p_a_pos_b_pos partition of p_a_pos for values from (0) to
(maxvalue);

prepare q1 (int, int) as select * from p where a = $1 and b = $1;

explain analyze execute q1 (-1,-1); -- this works.
QUERY PLAN
--------------------------------------------------------------------------------------------------------------
Append (cost=0.00..175.60 rows=4 width=8) (actual time=1.099..1.099
rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_neg (cost=0.00..43.90 rows=1 width=8)
(actual time=0.023..0.023 rows=0 loops=1)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_neg (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
(12 rows)

explain analyze execute q1 (-1,1); -- should scan p_a_neg_b_pos, but does not.
QUERY PLAN
--------------------------------------------------------------------------------------------------------------
Append (cost=0.00..175.60 rows=4 width=8) (actual
time=758996.359..758996.359 rows=0 loops=1)
Runtime Partition Pruning: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_neg (cost=0.00..43.90 rows=1 width=8)
(actual time=0.056..0.056 rows=0 loops=1)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_neg_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_neg (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
-> Seq Scan on p_a_pos_b_pos (cost=0.00..43.90 rows=1 width=8)
(never executed)
Filter: ((a = $1) AND (b = $1))
(12 rows)

So, I started to look at what the best way to put this right might be.
I see that since Parallel Append was committed that the subnodes are
now sorted in cost order inside create_append_path(), so likely we'll
want to delay figuring out the subpath list indexes until after that's
done since sorting would scramble our index arrays. We could simply
look at the subpaths at the end of create_append_path() and create
some sort of new matrix type that can accept the output of Amit's
get_partitions_from_clauses() and translate that Bitmapset into the
subpath indexes (another Bitmapset). This will also need to work for
sub-partitions too, so this matrix must be some sort of tree that we
can descend into when we see that get_partitions_from_clauses returned
a bit for a sub-partition instead of a leaf-partition.

Yes, the change in sort order means that the current
append_paths_array cannot be used for Parallel append and a new logic
has to be devised. I have still not thought about it but your method
seems like a good way to go. Currently I have worked on the Parallel
bit considering that the appends_path_array holds the correct
subplan_index.

I bashed this idea around a bit and I came up with the attached. It's
very far from complete and in a very WIP state. I've not really done
anything to make the correct clause list available in nodeAppend.c
yet, but I think the code that's there is worthy of a look. I've not
done that much work on the new choose_next_subplan* functions in
nodeAppend.c. I just modified choose_next_subplan_locally to show how
this set of functions need to take into account the subnode bitmap set
of valid partitions to scan. Perhaps some special case is needed to
have these functions ignore the Bitmapset when runtime pruning is
disabled (perhaps a completely new set of the functions is needed to
support the selection of the next non-pruned partition). Although,
probably that can be debated a bit later as it's a fairly minor detail
for now.

My patch also lacks any means to extract the Params during
match_clauses_to_partkey(), or at least most of the cases. I've just
added 1 case there. I did this because I thought it was better to
extract the ParamIds rather than a bool to say we've matched params.
This way we can only reevaluate which subplans to look at on rescan of
an Append if and only if the params we actually care about have
changed. I've not given this part a huge amount of thought yet.

Currently, during ReScan the patch prunes whenever the Param changes
and I had this 'rescan pruning optimization' in mind but had not
worked on it.

I'm a little unsure where to go from here. Obviously, this is quite a
major rewrite of your patch. The parts that I've got missing likely
can use quite a bit of the stuff you've already written, but that
needs some review. I wanted to post this now as I know you're busy
working on this to rebase it on parallel Append and to also address
Robert's concerns, which all seem valid to me.

What's the best way for us to coordinate our efforts on this? Maybe
you could look at my patch and sanity check it to ensure I'm not
taking anything in the wrong direction?

I have not seen the patch in depth but this approach seems good. Dilip
has already worked on the join equality bug you pointed out before, I
am yet to merge that patch and will work on the optimizer comments of
Robert's. Maybe I can incorporate your patch while working on it.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#42Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#40)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Mon, Dec 18, 2017 at 4:03 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

We could do something similar here using a similar code structure. Maybe,
add a ExecSetupPartitionRuntimePruning() in execPartition.c (mimicking
ExecSetupPartitionTupleRouting), that accepts AppendState node.
Furthermore, it might be a good idea to have something similar to
ExecFindPartition(), say, ExecGetPartitions(). That is, we have new
functions for run-time pruning that are counterparts to corresponding
functions for tuple routing.

Seems to me in this case we're better to build this structure during
planning and save it with the plan so that it can be used over and
over, rather than building it again and again each time the plan is
executed. Likely a common use case for run-time pruning is when the
plan is going to be used multiple times with different parameters, so
we really don't want to repeat any work that we don't have to here.

I agree. It would be better to avoid building the structure during execution.
PFA the updated patch.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

0001-Implement-runtime-partiton-pruning_v7_WIP.patchapplication/octet-stream; name=0001-Implement-runtime-partiton-pruning_v7_WIP.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 7e3a777..e0c818a 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,6 +34,7 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
@@ -279,11 +280,14 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+									int rt_index, List *clauses,
+									ParamListInfo base_prmlist,
+									ExprContext *econtext);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
 						 List **result, bool *constfalse);
@@ -293,7 +297,8 @@ static bool partition_cmp_args(PartitionKey key, int partattoff,
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 
@@ -1663,6 +1668,37 @@ get_partition_qual_relid(Oid relid)
 	return result;
 }
 
+/* get_leaf_part_recurse
+ *		Get the leaf oids for the given rel.
+ */
+void
+get_leaf_part_recurse(Relation rel, List **leaf_part_oids)
+{
+	PartitionDesc partdesc = RelationGetPartitionDesc(rel);
+	int			i;
+
+	check_stack_depth();
+
+	for (i = 0; i < partdesc->nparts; i++)
+	{
+		Oid			partrelid = partdesc->oids[i];
+
+		if (get_rel_relkind(partrelid) != RELKIND_PARTITIONED_TABLE)
+			*leaf_part_oids = lappend_oid(*leaf_part_oids, partrelid);
+		else
+		{
+			/*
+			 * We assume all tables in the partition tree were already locked
+			 * by the caller.
+			 */
+			Relation	partrel = heap_open(partrelid, NoLock);
+
+			get_leaf_part_recurse(partrel, leaf_part_oids);
+			heap_close(partrel, NoLock);
+		}
+	}
+}
+
 /*
  * get_partitions_from_clauses
  *		Determine the set of partitions of relation that will satisfy all
@@ -1673,7 +1709,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1707,7 +1744,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 	}
 
 	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+												 partclauses, base_prmlist,
+												 econtext);
 
 	return result;
 }
@@ -1723,7 +1761,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
-									List *clauses)
+									List *clauses, ParamListInfo base_prmlist,
+									ExprContext *econtext)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
 	Bitmapset *result = NULL;
@@ -1739,7 +1778,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
 											 &keys, &constfalse,
-											 &or_clauses);
+											 &or_clauses, base_prmlist,
+											 econtext);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1799,7 +1839,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 
 			arg_partset = get_partitions_from_clauses_recurse(relation,
 															  rt_index,
-															  arg_clauses);
+															  arg_clauses,
+															  base_prmlist,
+															  econtext);
 
 			/*
 			 * Partition sets obtained from mutually-disjunctive clauses are
@@ -1857,7 +1899,8 @@ static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
 								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses)
+								 List **or_clauses, ParamListInfo base_prmlist,
+								 ExprContext *econtext)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int		i;
@@ -1897,6 +1940,9 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				continue;
 			}
 		}
+		/* when called from ExecReScanAppend */
+		else if (IsA(lfirst(lc), ExprState))
+			clause = ((ExprState *) lfirst(lc))->expr;
 		else
 			clause = (Expr *) lfirst(lc);
 
@@ -2332,7 +2378,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
-										&keys->maxkeys[i]))
+										&keys->maxkeys[i], base_prmlist,
+										econtext))
 			{
 				keys->n_maxkeys++;
 				keys->max_incl = incl;
@@ -2344,12 +2391,14 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->eqkeys[i]))
+											&keys->eqkeys[i], base_prmlist,
+											econtext))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->maxkeys[i]))
+											&keys->maxkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_maxkeys++;
 					keys->max_incl = true;
@@ -2357,7 +2406,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
-											&keys->minkeys[i]))
+											&keys->minkeys[i], base_prmlist,
+											econtext))
 				{
 					keys->n_minkeys++;
 					keys->min_incl = true;
@@ -2365,7 +2415,8 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
-											 &keys->minkeys[i]))
+											 &keys->minkeys[i], base_prmlist,
+											 econtext))
 			{
 				keys->n_minkeys++;
 				keys->min_incl = incl;
@@ -2453,7 +2504,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, Datum *value, ParamListInfo base_prmlist,
+						ExprContext *econtext)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2496,6 +2548,46 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (base_prmlist)
+					{
+						Node	   *n;
+
+						n = eval_const_expressions_from_list(base_prmlist,
+															 (Node *) expr);
+
+						if (IsA(n, Const))
+						{
+							*value = ((Const *) n)->constvalue;
+							return true;
+						}
+					}
+					return false;
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
+			return false;
+
 		default:
 			return false;
 	}
@@ -2758,10 +2850,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, &leftarg_const,
+								 NULL, NULL))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, &rightarg_const,
+								 NULL, NULL))
 		return false;
 
 	/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 7e4fbaf..e3ac4ac 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -1347,6 +1347,21 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	if (es->format == EXPLAIN_FORMAT_TEXT)
 		appendStringInfoChar(es->str, '\n');
 
+	/* run-time pruning information for Append node */
+	if (es->analyze && IsA(plan, Append))
+	{
+		Append	   *append = (Append *) plan;
+
+		if (append->base_params)
+			show_scan_qual(append->base_params, "Runtime Partition Pruning",
+						   planstate, ancestors, es);
+
+		if (append->join_clauses)
+			show_scan_qual(append->join_clauses,
+						   "Runtime Partition Pruning Join Filter",
+						   planstate, ancestors, es);
+	}
+
 	/* target list */
 	if (es->verbose)
 		show_plan_tlist(planstate, ancestors, es);
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index d545af2..b6872a2 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -23,8 +23,6 @@
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
 
-static PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
-								 int *num_parted, List **leaf_part_oids);
 static void get_partition_dispatch_recurse(Relation rel, Relation parent,
 							   List **pds, List **leaf_part_oids);
 static void FormPartitionKeyDatum(PartitionDispatch pd,
@@ -284,7 +282,7 @@ ExecFindPartition(ResultRelInfo *resultRelInfo, PartitionDispatch *pd,
  * All the relations in the partition tree (including 'rel') must have been
  * locked (using at least the AccessShareLock) by the caller.
  */
-static PartitionDispatch *
+PartitionDispatch *
 RelationGetPartitionDispatchInfo(Relation rel,
 								 int *num_parted, List **leaf_part_oids)
 {
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..d089fc3 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,15 +57,20 @@
 
 #include "postgres.h"
 
+#include "catalog/pg_inherits_fn.h"
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include <unistd.h>
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
 {
 	LWLock		pa_lock;		/* mutual exclusion to choose next subplan */
+	LWLock		prune_lock;		/* mutual exclusion to choose next subplan */
 	int			pa_next_plan;	/* next plan to choose by any worker */
+	bool		is_runtime_pruned;
 
 	/*
 	 * pa_finished[i] should be true if no more workers should select subplan
@@ -84,6 +89,85 @@ static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
 
 /* ----------------------------------------------------------------
+ *		set_append_subplan_indexes
+ *
+ *		Determine the subplans to scan based on the clauses.
+ * ----------------------------------------------------------------
+ */
+static void
+set_append_subplan_indexes(PlanState *pstate, AppendState *node, List *clauses)
+{
+	ParallelAppendState *as_pstate = node->as_pstate;
+	Append	   *plan = (Append *) pstate->plan;
+	Bitmapset  *partset = NULL,
+			   *subplans = NULL;
+	PartitionDispatch *pd = (PartitionDispatch *) plan->pd,
+				parent,
+			   *p1;
+	List	   *parents = NIL;
+	int			cur_index,
+				i;
+
+	parents = lappend(parents, &pd[0]);
+	node->subplan_indexes = NIL;
+	do
+	{
+		p1 = linitial(parents);
+		parent = *p1;
+
+		partset = get_partitions_from_clauses(parent->reldesc,
+											  1,
+											  list_copy(clauses),
+											  pstate->state->es_param_list_info,
+											  node->ps.ps_ExprContext);
+
+		if (!bms_is_empty(partset))
+		{
+			while ((cur_index = bms_first_member(partset)) >= 0)
+			{
+				if (parent->indexes[cur_index] >= 0)
+					subplans =
+						bms_add_member(subplans, parent->indexes[cur_index]);
+				else
+					parents =
+						lappend(parents, &pd[-parent->indexes[cur_index]]);
+			}
+		}
+		parents = list_delete_first(parents);
+	} while (parents);
+
+	if (node->ps.plan->parallel_aware)
+	{
+		LWLockAcquire(&as_pstate->pa_lock, LW_EXCLUSIVE);
+		for (i = 0; i < node->as_nplans; i++)
+			as_pstate->pa_finished[i] = true;
+		LWLockRelease(&as_pstate->pa_lock);
+	}
+
+	if (!bms_is_empty(subplans))
+	{
+		while ((i = bms_first_member(subplans)) >= 0)
+		{
+			int			index = plan->append_paths_array[i];
+
+			if (index >= 0)
+			{
+				node->subplan_indexes = lappend_int(node->subplan_indexes,
+													index);
+
+				if (node->ps.plan->parallel_aware)
+				{
+					LWLockAcquire(&as_pstate->pa_lock, LW_EXCLUSIVE);
+					node->as_pstate->pa_finished[index] = false;
+					LWLockRelease(&as_pstate->pa_lock);
+				}
+			}
+		}
+	}
+
+}
+
+/* ----------------------------------------------------------------
  *		ExecInitAppend
  *
  *		Begin all of the subscans of the append node.
@@ -127,17 +211,19 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->subplan_indexes = NIL;
+	appendstate->as_whichpartition = -1;
 
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
 
+	/* create expression context for node */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * append nodes have Result slots, which hold pointers to tuples, so we
+	 * have to initialize them.
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
@@ -154,6 +240,33 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		i++;
 	}
 
+	node->join_exprstates = ExecInitExprList(node->join_clauses,
+													 (PlanState *) appendstate);
+
+	if (node->base_params || node->join_exprstates)
+	{
+		Relation	rel;
+		PartitionDispatch *pd;
+		List	   *leaf_part_oids = NIL;
+		int			num_parted;
+
+		/*
+		 * Get the information about the partition tree after locking all the
+		 * partitions.
+		 */
+		rel = relation_open(node->parentoid, NoLock);
+		pd = RelationGetPartitionDispatchInfo(rel, &num_parted, &leaf_part_oids);
+		relation_close(rel, NoLock);
+		node->pd = pd;
+		for (i = 1; i < num_parted; i++)
+		{
+			PartitionDispatch partdispatch = pd[i];
+
+			heap_close(partdispatch->reldesc, NoLock);
+			ExecDropSingleTupleTableSlot(partdispatch->tupslot);
+		}
+	}
+
 	/*
 	 * initialize output tuple type
 	 */
@@ -184,12 +297,47 @@ static TupleTableSlot *
 ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
+	Append		*plan	= (Append *) pstate->plan;
+
+	/* Determine the subplans to scan depending on the base_params */
+	if (plan->base_params && node->as_whichpartition == -1)
+	{
+
+		if (node->ps.plan->parallel_aware)
+		{
+		ParallelAppendState *as_pstate = node->as_pstate;
+			LWLockAcquire(&as_pstate->prune_lock, LW_EXCLUSIVE);
+	
+			if(!as_pstate->is_runtime_pruned)
+			{
+				  set_append_subplan_indexes(pstate, node, plan->base_params);
+		            as_pstate->is_runtime_pruned = true;
+				if (node->subplan_indexes)
+					as_pstate->pa_next_plan = linitial_int(node->subplan_indexes);;
+			}
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+			LWLockRelease(&as_pstate->prune_lock);
+		}	
+		else
+		{
+			set_append_subplan_indexes(pstate, node, plan->base_params);
+			if (node->subplan_indexes)
+			{
+				node->as_whichplan = linitial_int(node->subplan_indexes);
+				node->as_whichpartition = 0;
+			}
+		}
+	}
+
+//	if (plan->is_runtime_prunable && node->as_whichpartition == -1)
+//		return ExecClearTuple(node->ps.ps_ResultTupleSlot);
 
 	/* If no subplan has been chosen, we must choose one before proceeding. */
 	if (node->as_whichplan == INVALID_SUBPLAN_INDEX &&
 		!node->choose_next_subplan(node))
 		return ExecClearTuple(node->ps.ps_ResultTupleSlot);
 
+
 	for (;;)
 	{
 		PlanState  *subnode;
@@ -256,6 +404,11 @@ void
 ExecReScanAppend(AppendState *node)
 {
 	int			i;
+	Append  *plan = node->ps.plan;
+
+	/* Determine subplans to scan based on the new Params */
+	if (node->ps.chgParam != NULL && plan->join_exprstates)
+		set_append_subplan_indexes((PlanState *) node, node, plan->join_exprstates);
 
 	for (i = 0; i < node->as_nplans; i++)
 	{
@@ -276,8 +429,21 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (node->subplan_indexes)
+	{
+		if (node->ps.plan->parallel_aware)
+			node->as_pstate->pa_next_plan = linitial_int(node->subplan_indexes);
+		node->as_whichplan = 
+			node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX :
+linitial_int(node->subplan_indexes);
+		node->as_whichpartition = 0;
+	}
+	else
+	{
+		node->as_whichplan =
+			node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+		node->as_whichpartition = -1;
+	}
 }
 
 /* ----------------------------------------------------------------
@@ -320,6 +486,7 @@ ExecAppendInitializeDSM(AppendState *node,
 	pstate = shm_toc_allocate(pcxt->toc, node->pstate_len);
 	memset(pstate, 0, node->pstate_len);
 	LWLockInitialize(&pstate->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+	LWLockInitialize(&pstate->prune_lock, LWTRANCHE_PARALLEL_APPEND);
 	shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, pstate);
 
 	node->as_pstate = pstate;
@@ -353,6 +520,7 @@ ExecAppendInitializeWorker(AppendState *node, ParallelWorkerContext *pwcxt)
 {
 	node->as_pstate = shm_toc_lookup(pwcxt->toc, node->ps.plan->plan_node_id, false);
 	node->choose_next_subplan = choose_next_subplan_for_worker;
+//	sleep(30);
 }
 
 /* ----------------------------------------------------------------
@@ -365,22 +533,50 @@ ExecAppendInitializeWorker(AppendState *node, ParallelWorkerContext *pwcxt)
 static bool
 choose_next_subplan_locally(AppendState *node)
 {
+	Append  *plan =  (Append *) node->ps.plan;
 	int			whichplan = node->as_whichplan;
 
 	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
 	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
 
+	/* Return false if no partition is selected by runtime pruning logic */
+	if (plan->is_runtime_prunable && node->as_whichpartition == -1)
+		return false;
+
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
 	{
-		if (whichplan >= node->as_nplans - 1)
+		/*
+		 * For runtime partition pruning, goto the next valid partition index
+		 */
+		if (plan->is_runtime_prunable)
+		{
+			if (++node->as_whichpartition < list_length(node->subplan_indexes))
+				node->as_whichplan = list_nth_int(node->subplan_indexes,
+												  node->as_whichpartition);
+			else
+				return false;
+		}
+
+		else if (whichplan >= node->as_nplans - 1)
 			return false;
-		node->as_whichplan++;
+		else
+			node->as_whichplan++;
 	}
 	else
 	{
 		if (whichplan <= 0)
 			return false;
-		node->as_whichplan--;
+
+		if (node->subplan_indexes)
+		{
+			if (--(node->as_whichpartition) >= 0)
+				node->as_whichplan = list_nth_int(node->subplan_indexes,
+												  node->as_whichpartition);
+			else
+				return false;
+		}
+		else
+			node->as_whichplan--;
 	}
 
 	return true;
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b1515dd..c46f439 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,12 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_SCALAR_FIELD(is_runtime_prunable);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(base_params);
+	COPY_NODE_FIELD(join_clauses);
+	COPY_SCALAR_FIELD(append_paths_size);
+	COPY_POINTER_FIELD(append_paths_array, from->append_paths_size * sizeof(int));
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b59a521..0f4c907 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -393,6 +393,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 static void
 _outAppend(StringInfo str, const Append *node)
 {
+	int i;
 	WRITE_NODE_TYPE("APPEND");
 
 	_outPlanInfo(str, (const Plan *) node);
@@ -400,6 +401,13 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(base_params);
+	WRITE_NODE_FIELD(join_clauses);
+	WRITE_OID_FIELD(parentoid);
+	WRITE_INT_FIELD(append_paths_size);
+	appendStringInfoString(str, " :append_paths_array");
+	for (i = 0; i < node->append_paths_size; i++)
+		appendStringInfo(str, " %d", node->append_paths_array[i]);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 0d17ae8..8dd2fe3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1601,7 +1601,11 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
-
+	READ_NODE_FIELD(base_params);
+	READ_NODE_FIELD(join_clauses);
+	READ_OID_FIELD(parentoid);
+	READ_INT_FIELD(append_paths_size);
+	READ_INT_ARRAY(append_paths_array, local_node->append_paths_size);
 	READ_DONE();
 }
 
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d222eff..609a7ca 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -25,6 +25,7 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_proc.h"
+#include "catalog/pg_inherits_fn.h"
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
@@ -144,9 +145,10 @@ static List *get_append_rel_partitions(PlannerInfo *root,
 static List *match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse);
-
+static int	list_member_oid_index(List *list, Oid datum);
 
 /*
  * make_one_rel
@@ -289,6 +291,27 @@ set_base_rel_sizes(PlannerInfo *root)
 		if (root->glob->parallelModeOK)
 			set_rel_consider_parallel(root, rel, rte);
 
+		if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			List	   *leaf_parts = NIL;
+			Relation	parent = relation_open(rte->relid, NoLock);
+			int			i;
+
+			/* fetch the leaf oids of the parent rel */
+			(void) find_all_inheritors(RelationGetRelid(parent), AccessShareLock, NULL);
+			get_leaf_part_recurse(parent, &leaf_parts);
+			relation_close(parent, NoLock);
+
+			/* initialize necessary root variables */
+			root->append_paths_size = list_length(leaf_parts);
+			root->leaf_node_oids = leaf_parts;
+			root->append_paths_array = palloc0(root->append_paths_size * sizeof(int));
+			root->baserestrictinfo_param_indexes = NIL;
+			root->append_paths_count = 0;
+
+			for (i = 0; i < root->append_paths_size; i++)
+				root->append_paths_array[i] = -1;
+		}
 		set_rel_size(root, rel, rti, rte);
 	}
 }
@@ -350,6 +373,40 @@ set_rel_size(PlannerInfo *root, RelOptInfo *rel,
 	{
 		/* It's an "append relation", process accordingly */
 		set_append_rel_size(root, rel, rti, rte);
+
+		/*
+		 * If this rel is part of a join then collect the necessary join
+		 * clauses required for runtime partition pruning.
+		 */
+		if (rel->joininfo && rel->part_scheme)
+		{
+			List	   *partclauses;
+			bool		contains_param,
+						contains_const,
+						constfalse;
+
+			/*
+			 * Get the clauses that match the partition key
+			 */
+			partclauses = match_clauses_to_partkey(root, rel,
+												   list_copy(rel->joininfo),
+												   &contains_param,
+												   &contains_const,
+												   &constfalse);
+
+			if (partclauses != NIL)
+			{
+				ListCell   *lc;
+
+				foreach(lc, partclauses)
+				{
+					Node	   *n = lfirst(lc);
+
+					if (!list_member(root->join_clauses, n))
+						root->join_clauses = lappend(root->join_clauses, n);
+				}
+			}
+		}
 	}
 	else
 	{
@@ -871,7 +928,8 @@ get_append_rel_partitions(PlannerInfo *root,
 {
 	List   *partclauses;
 	bool	contains_const,
-			constfalse;
+			constfalse,
+			contains_param;
 	List   *result = NIL;
 	int		i;
 	Relation		parent;
@@ -885,6 +943,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &contains_param,
 										   &contains_const,
 										   &constfalse);
 
@@ -901,7 +960,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	if (partclauses != NIL && contains_const)
 		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+												  partclauses, NULL, NULL);
 	else
 	{
 		/*
@@ -962,17 +1021,21 @@ get_append_rel_partitions(PlannerInfo *root,
  *
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
+ *
+ * If the list has a param, *contains_param is set
  */
 static List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 bool *contains_param,
 						 bool *contains_const,
 						 bool *constfalse)
 {
 	PartitionScheme	partscheme = rel->part_scheme;
 	List	   *result = NIL;
 	ListCell   *lc;
+	int			clause_index = -1;
 
 	*contains_const = false;
 	*constfalse = false;
@@ -985,6 +1048,8 @@ match_clauses_to_partkey(PlannerInfo *root,
 		Expr   *clause;
 		int		i;
 
+		clause_index++;
+
 		if (IsA(member, RestrictInfo))
 		{
 			RestrictInfo *rinfo = (RestrictInfo *) member;
@@ -1022,14 +1087,24 @@ match_clauses_to_partkey(PlannerInfo *root,
 				bool	arg_matches_key = false,
 						matched_arg_contains_const = false,
 						all_args_constfalse = true;
+				List	   *base_param_indexes;
+
+				/*
+				 * Make a copy of root's baserestrictinfo_param_indexes before
+				 * the recursive call
+				 */
+				base_param_indexes =
+					list_copy(root->baserestrictinfo_param_indexes);
 
 				foreach (lc1, orclause->args)
 				{
 					Node   *arg = lfirst(lc1);
-					bool	contains_const1,
-							constfalse1;
+					bool	contains_const1;
+					bool	constfalse1;
+					bool	contains_param1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 &contains_param1,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1040,8 +1115,22 @@ match_clauses_to_partkey(PlannerInfo *root,
 					/* We got at least one arg that is not constant false. */
 					if (!constfalse1)
 						all_args_constfalse = false;
+
+					/*
+					 * We got at least one arg which is Param, so add the
+					 * current clause_index to the base_param_indexes list.
+					 */
+					if (contains_param1)
+					{
+						*contains_param = true;
+						base_param_indexes =
+							list_append_unique_int(base_param_indexes,
+												   clause_index);
+					}
 				}
 
+				root->baserestrictinfo_param_indexes = base_param_indexes;
+
 				if (arg_matches_key)
 				{
 					result = lappend(result, clause);
@@ -1148,6 +1237,15 @@ match_clauses_to_partkey(PlannerInfo *root,
 				if (contain_volatile_functions((Node *) constexpr))
 					continue;
 
+				if (IsA(constexpr, Param))
+				{
+					*contains_param = true;
+
+					root->baserestrictinfo_param_indexes =
+						list_append_unique_int(root->baserestrictinfo_param_indexes,
+											   clause_index);
+				}
+
 				/*
 				 * Everything seems to be fine, so add it to the list of
 				 * clauses we will use for pruning.
@@ -1247,6 +1345,26 @@ match_clauses_to_partkey(PlannerInfo *root,
 	return result;
 }
 
+/* list_member_oid_index
+ *	  Returns the index of the given datum in the oid list.
+ *
+ *	It returns -1 if the datum is not found.
+ */
+static int
+list_member_oid_index(List *list, Oid datum)
+{
+	int			i = 0;
+	const ListCell *cell;
+
+	foreach(cell, list)
+	{
+		if (lfirst_oid(cell) == datum)
+			return i;
+		i++;
+	}
+	return -1;
+}
+
 /*
  * set_append_rel_size
  *	  Set size estimates for a simple "append relation"
@@ -1643,7 +1761,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 	foreach(l, rel_appinfos)
 	{
 		AppendRelInfo *appinfo = (AppendRelInfo *) lfirst(l);
-		int			childRTindex;
+		int			childRTindex,
+					index;
 		RangeTblEntry *childRTE;
 		RelOptInfo *childrel;
 
@@ -1676,6 +1795,11 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
 		if (IS_DUMMY_REL(childrel))
 			continue;
 
+		/* only consider non dummy children */
+		index = list_member_oid_index(root->leaf_node_oids, childRTE->relid);
+		if (index >= 0)
+			root->append_paths_array[index] = root->append_paths_count++;
+
 		/*
 		 * Child is live, so add it to the live_childrels list for use below.
 		 */
@@ -1911,7 +2035,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1953,7 +2077,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths, NULL,
 										parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
@@ -2002,7 +2126,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2058,7 +2182,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2323,7 +2447,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..26a5f6e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index f6c83d0..936a0b8 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1002,6 +1002,24 @@ create_join_plan(PlannerInfo *root, JoinPath *best_path)
 	return plan;
 }
 
+static List *
+replace_partition_nestloop_params(PlannerInfo *root, List *old_list)
+{
+	List	   *new_list = NIL;
+	ListCell   *lc;
+
+	if (old_list == NULL)
+		return NULL;
+
+	foreach(lc, old_list)
+	{
+		Node	   *n = lfirst(lc);
+
+		new_list = lappend(new_list, replace_nestloop_params(root, n));
+	}
+	return new_list;
+}
+
 /*
  * create_append_plan
  *	  Create an Append plan for 'best_path' and (recursively) plans
@@ -1065,6 +1083,14 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
+	plan->base_params = best_path->base_params;
+	plan->join_clauses = replace_partition_nestloop_params(root, best_path->join_clauses);
+	plan->parentoid = best_path->parentoid;
+	plan->parentrelid = best_path->path.parent->relid;
+	plan->append_paths_size = best_path->append_paths_size;
+	plan->append_paths_array = best_path->append_paths_array;
+	plan->is_runtime_prunable = (plan->join_clauses || plan->base_params);
+
 	return (Plan *) plan;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 7a09f07..565d5b7 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,7 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root, grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index a24e8ac..aab53dd 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ce83fbc..60f31e3 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 54126fb..7091a4f 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1269,6 +1270,56 @@ create_append_path(RelOptInfo *rel,
 	if (rows >= 0)
 		pathnode->path.rows = rows;
 
+	/* Do necessary evaluation needed for runtime partition pruning. */
+	if (root && (required_outer || rel->baserestrictinfo))
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		pathnode->append_paths_array = root->append_paths_array;
+		pathnode->append_paths_size = root->append_paths_size;
+
+		if (rte && rte->rtekind == RTE_RELATION)
+		{
+			Oid			poid = rte->relid;
+			Relation	prel = relation_open(poid, NoLock);
+
+			if (prel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+			{
+				if (root->join_clauses)
+					pathnode->join_clauses = root->join_clauses;
+
+				if (root->baserestrictinfo_param_indexes && rel->baserestrictinfo)
+				{
+					ListCell   *lc;
+					List	   *quals = list_copy(rel->baserestrictinfo);
+
+					pathnode->base_params = NIL;
+
+					/*
+					 * collect the clauses from baserestrictinfo for partition
+					 * pruning
+					 */
+					foreach(lc, root->baserestrictinfo_param_indexes)
+					{
+						int			index = lfirst_int(lc);
+						Node	   *member = list_nth(quals, index);
+						Expr	   *clause;
+
+						if (IsA(member, RestrictInfo))
+							clause = ((RestrictInfo *) member)->clause;
+						else
+							clause = (Expr *) member;
+
+						pathnode->base_params = lappend(pathnode->base_params,
+														clause);
+					}
+				}
+				pathnode->parentoid = poid;
+			}
+			relation_close(prel, NoLock);
+		}
+	}
+
 	return pathnode;
 }
 
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index dcfda1c..fc7cbcd 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -1554,6 +1554,9 @@ ParamPathInfo *
 get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 {
 	ParamPathInfo *ppi;
+	Relids		joinrelids;
+	List	   *pclauses;
+	ListCell   *lc;
 
 	/* Unparameterized paths have no ParamPathInfo */
 	if (bms_is_empty(required_outer))
@@ -1565,11 +1568,27 @@ get_appendrel_parampathinfo(RelOptInfo *appendrel, Relids required_outer)
 	if ((ppi = find_param_path_info(appendrel, required_outer)))
 		return ppi;
 
+	/*
+	 * To determine whether the appendrel is applicable for runtime pruning or
+	 * not, we fetch the clause from the join clause.
+	 */
+	joinrelids = bms_union(appendrel->relids, required_outer);
+	pclauses = NIL;
+	foreach(lc, appendrel->joininfo)
+	{
+		RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
+
+		if (join_clause_is_movable_into(rinfo,
+										appendrel->relids,
+										joinrelids))
+			pclauses = lappend(pclauses, rinfo);
+	}
+
 	/* Else build the ParamPathInfo */
 	ppi = makeNode(ParamPathInfo);
 	ppi->ppi_req_outer = required_outer;
 	ppi->ppi_rows = 0;
-	ppi->ppi_clauses = NIL;
+	ppi->ppi_clauses = pclauses;
 	appendrel->ppilist = lappend(appendrel->ppilist, ppi);
 
 	return ppi;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..c4a7788 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,7 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses);
+							List *partclauses, ParamListInfo base_prmlist,
+							ExprContext *econtext);
+extern void get_leaf_part_recurse(Relation rel, List **leaf_part_oids);
 #endif							/* PARTITION_H */
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 86a199d..e6fea65 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -49,6 +49,9 @@ typedef struct PartitionDispatchData
 
 typedef struct PartitionDispatchData *PartitionDispatch;
 
+extern PartitionDispatch *RelationGetPartitionDispatchInfo(Relation rel,
+								 int *num_parted, List **leaf_part_oids);
+
 extern void ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel,
 							   Index resultRTindex,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1a35c5c..f0ac6bc 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1006,6 +1006,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionDispatchData;
 
 struct AppendState
 {
@@ -1016,6 +1017,8 @@ struct AppendState
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
 	bool		(*choose_next_subplan) (AppendState *);
+	int			as_whichpartition;
+	List       *subplan_indexes;
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 02fb366..f04105e 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -19,6 +19,7 @@
 #include "nodes/bitmapset.h"
 #include "nodes/lockoptions.h"
 #include "nodes/primnodes.h"
+#include "nodes/relation.h"
 
 
 /* ----------------------------------------------------------------
@@ -242,6 +243,7 @@ typedef struct ModifyTable
  *		Generate the concatenation of the results of sub-plans.
  * ----------------
  */
+struct PartitionDispatchData;
 typedef struct Append
 {
 	Plan		plan;
@@ -249,6 +251,17 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/* for runtime partition pruning */
+	bool		is_runtime_prunable;
+	Oid			parentoid;
+	Index		parentrelid;
+	List	   *base_params;
+	List	   *join_clauses;
+	List	   *join_exprstates;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	struct PartitionDispatchData **pd;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index bcb669d..b7b6945 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -317,6 +317,17 @@ typedef struct PlannerInfo
 
 	/* optional private data for join_search_hook, e.g., GEQO */
 	void	   *join_search_private;
+
+	/* These fields accumulate data required for runtime partition pruning. */
+	int		   *append_paths_array; /* array for append subplans info */
+	int			append_paths_size;	/* size of append_paths_array */
+	int			append_paths_count; /* current count of append_paths_array */
+	List	   *leaf_node_oids; /* leaf oids of current rel */
+	List	   *baserestrictinfo_param_indexes; /* index from baserestrictinfo
+												 * list with partition related
+												 * clauses */
+	List	   *join_clauses;	/* join clause which are required for
+								 * partition pruning */
 } PlannerInfo;
 
 
@@ -1295,6 +1306,13 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	/* for runtime partition pruning */
+	Oid			parentoid;
+	int		   *append_paths_array;
+	int			append_paths_size;
+	List	   *base_params;
+	List	   *join_clauses;
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..f8e7660 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -79,6 +79,7 @@ extern void CommuteOpExpr(OpExpr *clause);
 extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist, Node *node);
 
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 00c134d..2ac3463 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,9 +64,9 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
-				   List *subpaths, List *partial_subpaths,
-				   Relids required_outer,
+extern AppendPath *create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel, List *subpaths,
+				   List *partial_subpaths, Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
 				   List *partitioned_rels, double rows);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
#43David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#42)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 19 December 2017 at 21:54, Beena Emerson <memissemerson@gmail.com> wrote:

PFA the updated patch.

Hi Beena,

Thanks for updating the patch. I've merged a chunk of your latest
patch with what I was working on and cleaned up some mistakes that I
had made in my WIP version.

I've now done the work to make this work with parallel append, which
wasn't so difficult. The 3 functions which choose the next subplan
just needed to be made aware that they need to pay attention to the
Bitmapset which lists which subplans need to be scanned. I did have to
write a bms_prev_member() function to get the next lower set bit in a
Bitmapset to make this easier. I think what I have is fairly clean.

I've also solved the problem we had discussed about only needing to
reselecting the subplans during ExecReScanAppend() when a parameter
has changed that is actually used in run-time pruning. If it's not
used, we now no longer wastefully reselect the same matching subplans.

The patch still does need more work in match_clauses_to_partkey() to
ensure we extract all paramids from the clauses. Right now, I'm only
grabbing the paramids from an OpExpr. There's a bunch of other places
Params could be hidden in there. Anything Amit is supporting for
Consts we need to grab the Param Ids for when it's a Param instead of
a Const.

There's also a few things about the patch which I didn't change around
too much as I didn't want change Amit's v15 patch too much. He working
on this still and I didn't want too many conflicts. Basically, I had
to make match_clauses_to_partkey() an external function so that I
could use it in createplan.c in create_append_plan() in order to
extract the Param Ids of any parameters in the parameterized path
clause. This function is currently in allpaths.c in Amit's patch, but
I'm feeling that it does not really belong there. I'll discuss with
Amit on the faster partition pruning thread.

The attached patch is pretty fresh. I've not given it a huge amount of
testing so far. I do know there's still work to do in order to make
cases like;

prepare q2 (int) as select * from p where a in(10000,20000,$1);
explain (costs off, analyze) execute q2 (45678);

work correctly. To fix that requires a bit more invasion into the
faster partition pruning v15 patch which this is based on. I don't
really want to touch that too much just yet.

Here are some examples of what the patch does:

-- Test 1 Setup
-- This shows the patch working with multiple partition levels.

drop table if exists pt;
create table pt (a int, b int) partition by range (a);

create table pt_a_neg partition of pt for values from (minvalue) to
(0) partition by range (b);
create table pt_a_pos partition of pt for values from (0) to
(maxvalue) partition by range (b);

create table pt_a_neg_b_neg partition of pt_a_neg for values from
(minvalue) to (0);
create table pt_a_neg_b_pos partition of pt_a_neg for values from (0)
to (maxvalue);

create table pt_a_pos_b_neg partition of pt_a_pos for values from
(minvalue) to (0);
create table pt_a_pos_b_pos partition of pt_a_pos for values from (0)
to (maxvalue);

insert into pt select x,x from generate_series(-1000,1000) x;

analyze pt;

prepare q1 (int, int) as select * from pt where a = $1 and b = $2;

-- Test 1 Result (first 5 custom plan executions removed)

postgres=# explain (costs off, analyze) execute q1 (-10,10);
QUERY PLAN
----------------------------------------------------------------------------
Append (actual time=0.007..0.007 rows=0 loops=1)
-> Seq Scan on pt_a_neg_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_neg_b_pos (actual time=0.007..0.007 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.366 ms
Execution time: 0.077 ms
(11 rows)

postgres=# explain (costs off, analyze) execute q1 (-10,-10);
QUERY PLAN
----------------------------------------------------------------------------
Append (actual time=0.235..0.237 rows=1 loops=1)
-> Seq Scan on pt_a_neg_b_neg (actual time=0.234..0.236 rows=1 loops=1)
Filter: ((a = $1) AND (b = $2))
Rows Removed by Filter: 999
-> Seq Scan on pt_a_neg_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.025 ms
Execution time: 0.313 ms
(12 rows)

postgres=# explain (costs off, analyze) execute q1 (10,-10);
QUERY PLAN
----------------------------------------------------------------------------
Append (actual time=0.014..0.014 rows=0 loops=1)
-> Seq Scan on pt_a_neg_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_neg_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_neg (actual time=0.013..0.013 rows=0 loops=1)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
Planning time: 0.025 ms
Execution time: 0.091 ms
(11 rows)

postgres=# explain (costs off, analyze) execute q1 (10,10);
QUERY PLAN
----------------------------------------------------------------------------
Append (actual time=0.032..0.222 rows=1 loops=1)
-> Seq Scan on pt_a_neg_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_neg_b_pos (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_neg (never executed)
Filter: ((a = $1) AND (b = $2))
-> Seq Scan on pt_a_pos_b_pos (actual time=0.031..0.221 rows=1 loops=1)
Filter: ((a = $1) AND (b = $2))
Rows Removed by Filter: 1000
Planning time: 0.026 ms
Execution time: 0.297 ms
(12 rows)

-- Test 2 Setup
-- This test shows the patch working with parameterized paths. Note
that only subplans which actually have a value matching a value from
the outer side of the join get scanned.

create table p (a int not null) partition by range(a);
select 'create table p'||x|| ' partition of p for values from ('||x *
10000 || ') to (' || (x+1)*10000 || ');' from generate_Series(0,9)x;
\gexec

insert into p select generate_Series(0,99999);

select 'create index on p'||x||' (a)' from generate_Series(0,9)x;
\gexec

create table t (a int not null);

insert into t values(4),(5);

analyze p,t;
set enable_mergejoin=0;
explain (costs off, analyze) select * from t inner join p p on p.a=t.a;
insert into t values(45678);
explain (costs off, analyze) select * from t inner join p p on p.a=t.a;

-- Test 2 Result
postgres=# explain (costs off, analyze) select * from t inner join p p
on p.a=t.a;
QUERY PLAN
----------------------------------------------------------------------------------------------
Nested Loop (actual time=0.058..0.069 rows=2 loops=1)
-> Seq Scan on t (actual time=0.011..0.012 rows=2 loops=1)
-> Append (actual time=0.020..0.021 rows=1 loops=2)
-> Index Only Scan using p0_a_idx on p0 p (actual
time=0.019..0.020 rows=1 loops=2)
Index Cond: (a = t.a)
Heap Fetches: 2
-> Index Only Scan using p1_a_idx on p1 p_1 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p2_a_idx on p2 p_2 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p3_a_idx on p3 p_3 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p4_a_idx on p4 p_4 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p5_a_idx on p5 p_5 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p6_a_idx on p6 p_6 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p7_a_idx on p7 p_7 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p8_a_idx on p8 p_8 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p9_a_idx on p9 p_9 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
Planning time: 1.080 ms
Execution time: 0.315 ms
(35 rows)

postgres=# insert into t values(45678); -- add a record to show that
p4 gets scanned.
INSERT 0 1
postgres=# explain (costs off, analyze) select * from t inner join p p
on p.a=t.a;
QUERY PLAN
------------------------------------------------------------------------------------------------
Nested Loop (actual time=0.041..0.106 rows=3 loops=1)
-> Seq Scan on t (actual time=0.014..0.014 rows=3 loops=1)
-> Append (actual time=0.024..0.025 rows=1 loops=3)
-> Index Only Scan using p0_a_idx on p0 p (actual
time=0.010..0.011 rows=1 loops=2)
Index Cond: (a = t.a)
Heap Fetches: 2
-> Index Only Scan using p1_a_idx on p1 p_1 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p2_a_idx on p2 p_2 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p3_a_idx on p3 p_3 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p4_a_idx on p4 p_4 (actual
time=0.049..0.050 rows=1 loops=1)
Index Cond: (a = t.a)
Heap Fetches: 1
-> Index Only Scan using p5_a_idx on p5 p_5 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p6_a_idx on p6 p_6 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p7_a_idx on p7 p_7 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p8_a_idx on p8 p_8 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
-> Index Only Scan using p9_a_idx on p9 p_9 (never executed)
Index Cond: (a = t.a)
Heap Fetches: 0
Planning time: 0.635 ms
Execution time: 0.233 ms
(35 rows)

There is, however, still a fundamental problem with the patch, or
maybe the idea in general (this was also pointed out by Amit Langote
in an off-list discussion):

The problem is down to the logic in choose_custom_plan() only choosing
a generic plan if the average cost of the generic plan is less than
the average custom plan cost. The problem is that the generic plan can
have many extra Append subnodes in comparison to the custom plan, all
of which are taken into account in the total plan cost, but these may
be pruned during execution. The logic in choose_custom_plan() has no
idea about this. I don't have any bright ideas on how to fix this
yet, as, suppose a PREPAREd statement like the following comes along:

PREPARE q3 (int, int) AS SELECT * FROM partitioned_table WHERE partkey
BETWEEN $1 AND $2;

the run-time pruning may prune it down no subplans, all subplans, or
any number in between. So we can't do anything like take the total
Append cost to be the highest costing of its subplans, and likely
using the average cost might not be a good idea either. It might work
sometimes, but likely won't be very stable. If this is not fixed then
choose_custom_plan() has a very low probability of choosing a generic
plan which has run-time partition pruning enabled, which in a way
defeats the purpose of this whole patch.

I'm a bit uncertain on the best way to resolve this. It needs to be
discussed here.

One more thing. The attached is not yet set up to work with
MergeAppend. It's likely just a small amount of additional work to
make this happen, so likely should be something that we do.

Anyway, I've attached the latest version of the patch. This is based
on Amit's v15 of faster-partition-pruning [1]/messages/by-id/06cde8a5-0ac7-dcf5-ad66-1ca781623e0c@lab.ntt.co.jp which I found to cleanly
apply to f94eec490

[1]: /messages/by-id/06cde8a5-0ac7-dcf5-ad66-1ca781623e0c@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v2.patchapplication/octet-stream; name=runtime_prune_drowley_v2.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 22de48a..7d04872 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -279,21 +281,26 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
-								 List **or_clauses);
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse, List **or_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext, bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1680,7 +1687,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							ParamListInfo prmlist, ExprContext *econtext,
+							 List *partclauses)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1713,12 +1721,128 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that's already been pruned. Set them all to
+	 * this initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths who's rel is not directly parented by rel.  We'll
+		 * process any we skip here later when looping through partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subnode.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the tables returned by get_partitions_from_clauses may be other
+	 * partitioned tables.  Unlike the case above, these won't be subnodes of
+	 * the Append.  To handle these we must make use of another
+	 * PartitionPruneInfo which we must use to recursively search until we
+	 * reach the leaf partitions.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the relation being queried.  We
+		 * only care about sub-partition parents here.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel. We'll search until we get down to the leaf partitions by
+		 * recursively calling ourself.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1730,6 +1854,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1745,8 +1871,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1806,6 +1932,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 
 			arg_partset = get_partitions_from_clauses_recurse(relation,
 															  rt_index,
+															  prmlist,
+															  econtext,
 															  arg_clauses);
 
 			/*
@@ -1863,8 +1991,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
  */
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
-								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 int rt_index, PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses)
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
@@ -2307,7 +2437,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2348,6 +2478,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
+										prmlist, econtext,
 										&keys->maxkeys[i]))
 			{
 				keys->n_maxkeys++;
@@ -2360,11 +2491,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->eqkeys[i]))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->maxkeys[i]))
 				{
 					keys->n_maxkeys++;
@@ -2373,6 +2506,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->minkeys[i]))
 				{
 					keys->n_minkeys++;
@@ -2381,6 +2515,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
+											 prmlist, econtext,
 											 &keys->minkeys[i]))
 			{
 				keys->n_minkeys++;
@@ -2469,7 +2604,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist, ExprContext *econtext,
+						Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2511,6 +2647,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2529,6 +2702,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2572,7 +2746,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2627,7 +2801,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2683,7 +2857,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2713,7 +2887,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2731,7 +2905,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2765,6 +2939,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -2774,10 +2949,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..7a4dbd9 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,6 +57,7 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
@@ -82,6 +83,9 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node,
+						   PartitionPruneInfo *pinfo);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +131,27 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
+
+	/*
+	 * Miscellaneous initialization
+	 *
+	 * create expression context for node
+	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
+	/*
+	 * The presence  of a part_prune_info means that run-time pruning is
+	 * enabled, so here we'll determine which subplans need to be scanned.
+	 */
+	if (node->part_prune_info)
+		set_valid_runtime_subplans(appendstate, appendstate->part_prune_info);
+
+	/* else, mark all subplans as requiring a scan */
+	else
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
 
 	/*
 	 * Miscellaneous initialization
@@ -163,10 +188,20 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Parallel-aware append plans must choose the first subplan to execute by
 	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
+	 * always start with the first valid subplan.
 	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (appendstate->ps.plan->parallel_aware)
+		appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		appendstate->as_whichplan = bms_next_member(
+										appendstate->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (appendstate->as_whichplan < 0)
+			appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +292,22 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * The presence  of a part_prune_info means that run-time pruning is
+	 * enabled.  If any of the parameters being used for partition pruning
+	 * have changed, then we'd better redetermine which subplans we need to
+	 * scan.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		/* Free previous valid subplan bits, if any */
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+
+		set_valid_runtime_subplans(node, node->part_prune_info);
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +327,17 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (node->ps.plan->parallel_aware)
+		node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		node->as_whichplan = bms_next_member(node->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (node->as_whichplan < 0)
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +426,19 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
 	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
 	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -399,6 +456,7 @@ choose_next_subplan_for_leader(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -412,21 +470,31 @@ choose_next_subplan_for_leader(AppendState *node)
 	}
 	else
 	{
-		/* Start with last subplan. */
-		node->as_whichplan = node->as_nplans - 1;
+		/* Start with last valid subplan. */
+		node->as_whichplan = bms_prev_member(validplans, -1);
+
+		/* Bail if there are no valid plans */
+		if (node->as_whichplan < 0)
+		{
+			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+			LWLockRelease(&pstate->pa_lock);
+			return false;
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[node->as_whichplan])
 	{
-		if (node->as_whichplan == 0)
+		node->as_whichplan = bms_prev_member(validplans, node->as_whichplan);
+
+		if (node->as_whichplan < 0)
 		{
 			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
 			node->as_whichplan = INVALID_SUBPLAN_INDEX;
 			LWLockRelease(&pstate->pa_lock);
 			return false;
 		}
-		node->as_whichplan--;
 	}
 
 	/* If non-partial, immediately mark as finished. */
@@ -457,6 +525,7 @@ choose_next_subplan_for_worker(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -477,15 +546,26 @@ choose_next_subplan_for_worker(AppendState *node)
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[pstate->pa_next_plan])
 	{
-		if (pstate->pa_next_plan < node->as_nplans - 1)
+		int nextplan;
+
+		nextplan = bms_next_member(validplans, pstate->pa_next_plan);
+		if (nextplan >= 0)
 		{
-			/* Advance to next plan. */
-			pstate->pa_next_plan++;
+			/* Advance to next valid plan. */
+			pstate->pa_next_plan = nextplan;
 		}
 		else if (append->first_partial_plan < node->as_nplans)
 		{
-			/* Loop back to first partial plan. */
-			pstate->pa_next_plan = append->first_partial_plan;
+			/* Loop back to first valid partial plan. */
+			pstate->pa_next_plan = bms_next_member(validplans,
+											append->first_partial_plan - 1);
+
+			/*
+			 * Ensure we there is a valid first partial plan, if not then
+			 * we'll the next plan so that we finish the loop below.
+			 */
+			if (pstate->pa_next_plan < 0)
+				pstate->pa_next_plan = node->as_whichplan;
 		}
 		else
 		{
@@ -503,8 +583,9 @@ choose_next_subplan_for_worker(AppendState *node)
 	}
 
 	/* Pick the plan we found, and advance pa_next_plan one more time. */
-	node->as_whichplan = pstate->pa_next_plan++;
-	if (pstate->pa_next_plan >= node->as_nplans)
+	node->as_whichplan = pstate->pa_next_plan;
+	pstate->pa_next_plan = bms_next_member(validplans, pstate->pa_next_plan);
+	if (pstate->pa_next_plan < 0)
 	{
 		if (append->first_partial_plan < node->as_nplans)
 			pstate->pa_next_plan = append->first_partial_plan;
@@ -526,3 +607,51 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * get_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		'prune_qual'.  All subplans which provably cannot possibly have
+ *		matching records are eliminated and the remainder are set in the
+ *		AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node, PartitionPruneInfo *pinfo)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+	List		   *clauses = node->prune_qual;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			node->as_valid_subplans = bms_add_member(node->as_valid_subplans,
+													 pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans(node, pinfo->subpartindex[i]);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subnodes");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index ae30072..fb104bc 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * 1-bit in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b1515dd..14189c0 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2125,6 +2127,31 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5006,6 +5033,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeaf8fd..ee53c72 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1711,6 +1719,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	List	   *partitioned_rels = NIL;
 	RangeTblEntry *rte;
 	double		partial_rows = -1;
+	bool		trypartitionprune = false;
 
 	/*
 	 * AppendPath generated for partitioned tables must record the RT indexes
@@ -1735,7 +1744,19 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		rte = planner_rt_fetch(rel->relid, root);
 		if (rte->rtekind == RTE_RELATION &&
 			rte->relkind == RELKIND_PARTITIONED_TABLE)
-		partitioned_rels = rel->live_partitioned_rels;
+		{
+			partitioned_rels = rel->live_partitioned_rels;
+
+			/*
+			 * For base partitioned tables we'll try to see if we can perform
+			 * any run-time partition pruning.  We need to do a bit more work
+			 * later in planning to ensure we can enable it, so this just
+			 * allows a way to save doing that work in cases when we certainly
+			 * can't enable it.
+			 */
+			if (rel->reloptkind == RELOPT_BASEREL)
+				trypartitionprune = true;
+		}
 	}
 	else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
 	{
@@ -1907,9 +1928,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
-												  partitioned_rels, -1));
+												  partitioned_rels, -1,
+												  trypartitionprune));
 
 	/*
 	 * Consider an append of unordered, unparameterized partial paths.  Make
@@ -1949,10 +1971,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
-										partitioned_rels, -1);
+										partitioned_rels, -1,
+										trypartitionprune);
 
 		/*
 		 * Make sure any subsequent partial paths use the same row count
@@ -1998,10 +2021,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
-										partitioned_rels, partial_rows);
+										partitioned_rels, partial_rows,
+										trypartitionprune);
 		add_partial_path(rel, (Path *) appendpath);
 	}
 
@@ -2054,9 +2078,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
-										partitioned_rels, -1));
+										partitioned_rels, -1,
+										trypartitionprune));
 	}
 }
 
@@ -2319,8 +2344,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..15d1426 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,8 +1232,8 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index f6c83d0..b5092fb 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,54 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual = extract_actual_clauses(best_path->path.parent->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+			RangeTblEntry *rte;
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			rte = planner_rt_fetch(rel->relid, root);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the prams and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll know it needs to use it when it's
+		 * there.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1115,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5297,19 +5354,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ffdf9c5..6139a13 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,14 +3678,16 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
 								   0,
 								   false,
 								   NIL,
-								   -1);
+								   -1,
+								   false);
 			path->pathtarget = target;
 		}
 		else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index a24e8ac..cc08769 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,8 +590,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
 
@@ -702,8 +702,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ce83fbc..60f31e3 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 54126fb..23d333e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,11 +1211,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows)
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
 	ListCell   *l;
@@ -1224,8 +1227,20 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When using run-time partition pruning we need the clause list details
+	 * in in the param_info, get_appendrel_parampathinfo does not do this, so
+	 * when we're trying to enable run-time partition pruning we'll just call
+	 * get_baserel_parampathinfo instead.
+	 */
+	if (trypartitionprune)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -1248,6 +1263,8 @@ create_append_path(RelOptInfo *rel,
 									  append_startup_cost_compare);
 	}
 	pathnode->first_partial_path = list_length(subpaths);
+	pathnode->part_prune_params = rel->runtime_prune_params;
+	pathnode->trypartitionprune = trypartitionprune;
 	pathnode->subpaths = list_concat(subpaths, partial_subpaths);
 
 	foreach(l, subpaths)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index dcfda1c..e673ea8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 853c1f6..e83720f 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1053,7 +1053,14 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	 * Note that if generic_cost is -1 (indicating we've not yet determined
 	 * the generic plan cost), we'll always prefer generic at this point.
 	 */
-	if (plansource->generic_cost < avg_custom_cost)
+
+	/*
+	 * XXX fix this. Currently we tend to favor custom plans when plan
+	 * time partition pruning is performed since the generic plans will
+	 * tend to cost more since they'll not have eliminated the same
+	 * partitions. For testing only, we'll just use a generic plan here.
+	 */
+	/* if (plansource->generic_cost < avg_custom_cost) */
 		return false;
 
 	return true;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..9ff6685 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,9 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 3b62a97..b4ded9a 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1a35c5c..3d5601e 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -999,6 +999,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1013,8 +1014,12 @@ struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Bitmapset  *as_valid_subplans;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* Param Ids useful for partion pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c5b5115..b21ecfb 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 02fb366..2035479 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,10 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/* Details of how to prune surplus subplans or NULL when disabled. */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 074ae0a..19695bd 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,24 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * recursively to search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index bcb669d..5d567f9 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,10 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	/* ParamIds useful for subpath elimination during execution, or NULL */
+	Bitmapset  *part_prune_params;
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..4fb48b1 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 00c134d..b05cde4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,11 +64,12 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows);
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..b98d0b9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
#44Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#43)
Re: [HACKERS] Runtime Partition Pruning

Hello David,

On Thu, Dec 21, 2017 at 2:31 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 19 December 2017 at 21:54, Beena Emerson <memissemerson@gmail.com> wrote:

The problem is down to the logic in choose_custom_plan() only choosing
a generic plan if the average cost of the generic plan is less than
the average custom plan cost. The problem is that the generic plan can
have many extra Append subnodes in comparison to the custom plan, all
of which are taken into account in the total plan cost, but these may
be pruned during execution. The logic in choose_custom_plan() has no
idea about this. I don't have any bright ideas on how to fix this
yet, as, suppose a PREPAREd statement like the following comes along:

PREPARE q3 (int, int) AS SELECT * FROM partitioned_table WHERE partkey
BETWEEN $1 AND $2;

the run-time pruning may prune it down no subplans, all subplans, or
any number in between. So we can't do anything like take the total
Append cost to be the highest costing of its subplans, and likely
using the average cost might not be a good idea either. It might work
sometimes, but likely won't be very stable. If this is not fixed then
choose_custom_plan() has a very low probability of choosing a generic
plan which has run-time partition pruning enabled, which in a way
defeats the purpose of this whole patch.

I'm a bit uncertain on the best way to resolve this. It needs to be
discussed here.

I had mentioned this in the first mail on this thread that the generic
plan is always preferred and Robert said that it is not in scope of
this patch. Maybe we can start a new thread for this.

One more thing. The attached is not yet set up to work with
MergeAppend. It's likely just a small amount of additional work to
make this happen, so likely should be something that we do.

Anyway, I've attached the latest version of the patch. This is based
on Amit's v15 of faster-partition-pruning [1] which I found to cleanly
apply to f94eec490

Thank you for working on this. I will look into this and merge with
my current version of patch and Amit's v16 patches and post a new
patch soon.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#45David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#44)
Re: [HACKERS] Runtime Partition Pruning

On 21 December 2017 at 23:51, Beena Emerson <memissemerson@gmail.com> wrote:

On Thu, Dec 21, 2017 at 2:31 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 19 December 2017 at 21:54, Beena Emerson <memissemerson@gmail.com> wrote:

The problem is down to the logic in choose_custom_plan() only choosing
a generic plan if the average cost of the generic plan is less than
the average custom plan cost. The problem is that the generic plan can
have many extra Append subnodes in comparison to the custom plan, all
of which are taken into account in the total plan cost, but these may
be pruned during execution. The logic in choose_custom_plan() has no
idea about this..

I had mentioned this in the first mail on this thread that the generic
plan is always preferred and Robert said that it is not in scope of
this patch. Maybe we can start a new thread for this.

Sorry, I missed that. I knew you had modified choose_custom_plan()
too, but I didn't know the diagnosis of the problem had made its way
here yet.

I wonder if Robert understands the extent of the problem. The patch
will be useful to prune away partitions when the partitioned table is
on the inside of a parameterised nested loop join, but I think we'll
likely get some complaints about PREPARE statements always using a
custom plan. It's going to make this pretty hard to write regression
tests for unless we went and invent some GUC "prefer_generic_plan", or
"generic_plan_cost_multiplier" to provide a way to coax
choose_custom_plan() into not choosing a custom plan. I admit to not
having the answer to this but I don't think we should rule out
discussing possible fixes.

Thank you for working on this. I will look into this and merge with
my current version of patch and Amit's v16 patches and post a new
patch soon.

okay. I've just rebased mine on top of Amit's v16 now and will send
shortly. I already took some code from your v7 patch to make my v2
work. Which parts do you think still need to be merged?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#46Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#45)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Thu, Dec 21, 2017 at 6:01 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 21 December 2017 at 23:51, Beena Emerson <memissemerson@gmail.com> wrote:

On Thu, Dec 21, 2017 at 2:31 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 19 December 2017 at 21:54, Beena Emerson <memissemerson@gmail.com> wrote:

The problem is down to the logic in choose_custom_plan() only choosing
a generic plan if the average cost of the generic plan is less than
the average custom plan cost. The problem is that the generic plan can
have many extra Append subnodes in comparison to the custom plan, all
of which are taken into account in the total plan cost, but these may
be pruned during execution. The logic in choose_custom_plan() has no
idea about this..

I had mentioned this in the first mail on this thread that the generic
plan is always preferred and Robert said that it is not in scope of
this patch. Maybe we can start a new thread for this.

Sorry, I missed that. I knew you had modified choose_custom_plan()
too, but I didn't know the diagnosis of the problem had made its way
here yet.

I wonder if Robert understands the extent of the problem. The patch
will be useful to prune away partitions when the partitioned table is
on the inside of a parameterised nested loop join, but I think we'll
likely get some complaints about PREPARE statements always using a
custom plan. It's going to make this pretty hard to write regression
tests for unless we went and invent some GUC "prefer_generic_plan", or
"generic_plan_cost_multiplier" to provide a way to coax
choose_custom_plan() into not choosing a custom plan. I admit to not
having the answer to this but I don't think we should rule out
discussing possible fixes.

I have attached the regression tests I am currently using to check my
patch. It has few prepared statements using runtime pruning.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

Attachments:

runtime_pruning_testcases.patchapplication/octet-stream; name=runtime_pruning_testcases.patchDownload
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index aabb024..f6a9fd2 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1093,3 +1093,483 @@ explain (costs off) select * from boolpart where a is not unknown;
 (7 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list(a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as select * from ab where a BETWEEN $1 and $2 AND b <= $3;
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Runtime Partition Pruning: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(20 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                              QUERY PLAN                              
+----------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Runtime Partition Pruning: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(20 rows)
+
+-- runtime pruning after optimizer pruning
+deallocate ab_q1;
+prepare ab_q1 (int, int) as select a from ab where a BETWEEN $1 and $2 AND b < 3;
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Runtime Partition Pruning: ((a >= $1) AND (a <= $2))
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(14 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Runtime Partition Pruning: ((a >= $1) AND (a <= $2))
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(14 rows)
+
+-- join
+DROP TABLE IF EXISTS tbl1;
+NOTICE:  table "tbl1" does not exist, skipping
+CREATE TABLE tbl1(col1 int);
+INSERT INTO tbl1 VALUES (501), (505);
+--basic table
+DROP TABLE IF EXISTS tprt;
+NOTICE:  table "tprt" does not exist, skipping
+CREATE TABLE tprt (col1 int) PARTITION BY range(col1);
+CREATE TABLE tprt_1 PARTITION OF tprt FOR VALUES FROM (1) TO (501);
+CREATE TABLE tprt_2 PARTITION OF tprt FOR VALUES FROM (501) TO (1001);
+CREATE TABLE tprt_3 PARTITION OF tprt FOR VALUES FROM (1001) TO (2001);
+CREATE TABLE tprt_4 PARTITION OF tprt FOR VALUES FROM (2001) TO (3001);
+CREATE TABLE tprt_5 PARTITION OF tprt FOR VALUES FROM (3001) TO (4001);
+CREATE TABLE tprt_6 PARTITION OF tprt FOR VALUES FROM (4001) TO (5001);
+CREATE INDEX tprt1_idx ON tprt_1 (col1);
+CREATE INDEX tprt2_idx ON tprt_2 (col1);
+CREATE INDEX tprt3_idx ON tprt_3 (col1);
+CREATE INDEX tprt4_idx ON tprt_4 (col1);
+CREATE INDEX tprt5_idx ON tprt_5 (col1);
+CREATE INDEX tprt6_idx ON tprt_6 (col1);
+INSERT INTO tprt VALUES (10), (20), (501), (502), (505), (1001), (4500);
+SET enable_hashjoin = OFF;
+SET enable_mergejoin = OFF;
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 > col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(22 rows)
+
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 = col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(22 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+--multiple partitions
+INSERT INTO tbl1 VALUES (1001), (1010);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=17 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=4 loops=4)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 > col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=3)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(22 rows)
+
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=1 loops=4)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 = col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(22 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+(17 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+--last partition
+DELETE FROM tbl1;
+INSERT INTO tbl1 VALUES (4400);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 < col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(22 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 < tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+--no partition
+DELETE FROM tbl1;
+INSERT INTO tbl1 VALUES (10000);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         Runtime Partition Pruning Join Filter: (tbl1.col1 = col1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(22 rows)
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+-- parallel append
+deallocate ab_q1;
+prepare ab_q1 (int, int) as select avg(a) from ab where a BETWEEN $1 and $2 AND b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=2;
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Runtime Partition Pruning: ((a >= $1) AND (a <= $2))
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(25 rows)
+
+drop table if exists ab;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 514f8e5..2f92e52 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -153,3 +153,116 @@ explain (costs off) select * from boolpart where a is unknown;
 explain (costs off) select * from boolpart where a is not unknown;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list(a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as select * from ab where a BETWEEN $1 and $2 AND b <= $3;
+
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+-- runtime pruning after optimizer pruning
+deallocate ab_q1;
+prepare ab_q1 (int, int) as select a from ab where a BETWEEN $1 and $2 AND b < 3;
+
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- join
+DROP TABLE IF EXISTS tbl1;
+CREATE TABLE tbl1(col1 int);
+INSERT INTO tbl1 VALUES (501), (505);
+
+--basic table
+
+DROP TABLE IF EXISTS tprt;
+CREATE TABLE tprt (col1 int) PARTITION BY range(col1);
+CREATE TABLE tprt_1 PARTITION OF tprt FOR VALUES FROM (1) TO (501);
+CREATE TABLE tprt_2 PARTITION OF tprt FOR VALUES FROM (501) TO (1001);
+CREATE TABLE tprt_3 PARTITION OF tprt FOR VALUES FROM (1001) TO (2001);
+CREATE TABLE tprt_4 PARTITION OF tprt FOR VALUES FROM (2001) TO (3001);
+CREATE TABLE tprt_5 PARTITION OF tprt FOR VALUES FROM (3001) TO (4001);
+CREATE TABLE tprt_6 PARTITION OF tprt FOR VALUES FROM (4001) TO (5001);
+
+CREATE INDEX tprt1_idx ON tprt_1 (col1);
+CREATE INDEX tprt2_idx ON tprt_2 (col1);
+CREATE INDEX tprt3_idx ON tprt_3 (col1);
+CREATE INDEX tprt4_idx ON tprt_4 (col1);
+CREATE INDEX tprt5_idx ON tprt_5 (col1);
+CREATE INDEX tprt6_idx ON tprt_6 (col1);
+
+INSERT INTO tprt VALUES (10), (20), (501), (502), (505), (1001), (4500);
+
+SET enable_hashjoin = OFF;
+SET enable_mergejoin = OFF;
+
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1;
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+
+--multiple partitions
+INSERT INTO tbl1 VALUES (1001), (1010);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1;
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 > tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+
+--last partition
+DELETE FROM tbl1;
+INSERT INTO tbl1 VALUES (4400);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 < tprt.col1;
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 < tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+
+--no partition
+DELETE FROM tbl1;
+INSERT INTO tbl1 VALUES (10000);
+explain (analyze, costs off, summary off, timing off) SELECT * FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1;
+SELECT tbl1.col1, tprt.col1 FROM tbl1 JOIN tprt ON tbl1.col1 = tprt.col1 ORDER BY tbl1.col1, tprt.col1;
+
+
+-- parallel append
+deallocate ab_q1;
+prepare ab_q1 (int, int) as select avg(a) from ab where a BETWEEN $1 and $2 AND b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=2;
+
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+
+drop table if exists ab;
#47David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#43)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 21 December 2017 at 22:01, David Rowley <david.rowley@2ndquadrant.com> wrote:

I've attached the latest version of the patch. This is based
on Amit's v15 of faster-partition-pruning [1] which I found to cleanly
apply to f94eec490

Well, that went out of date pretty quickly. Amit has now posted v16 of
the faster partition pruning patch [1]/messages/by-id/5ebae4cf-8145-975c-ad75-16eb7f756f32@lab.ntt.co.jp which conflicts with my changes
in my v2 patch.

I've attached a new version of the patch to resolve these conflicts.

In v16 Amit added some code to eliminate LIST partitions when there
are not equals expressions which cause all the items in the IN list to
be unmatchable.

This appears to work fine with the run-time pruning in the attached, per:

create table ta (a int not null) partition by list (a);
create table ta1 partition of ta for values in(1,2);
create table ta2 partition of ta for values in(3,4);
explain select * from ta where a <> 1 and a <> 2;
create table ta_null partition of ta for values in(null);

prepare q1 (int) as select * from ta where a <> 1 and a <> $1;

explain (costs off, analyze) execute q1(2);
QUERY PLAN
-----------------------------------------------------------------
Append (actual time=0.012..0.012 rows=0 loops=1)
-> Seq Scan on ta1 (never executed)
Filter: ((a <> 1) AND (a <> $1))
-> Seq Scan on ta2 (actual time=0.012..0.012 rows=0 loops=1)
Filter: ((a <> 1) AND (a <> $1))
Planning time: 0.019 ms
Execution time: 0.057 ms
(7 rows)

explain (costs off, analyze) execute q1(1);
QUERY PLAN
-----------------------------------------------------------------
Append (actual time=0.017..0.017 rows=0 loops=1)
-> Seq Scan on ta1 (actual time=0.013..0.013 rows=0 loops=1)
Filter: ((a <> 1) AND (a <> $1))
-> Seq Scan on ta2 (actual time=0.003..0.003 rows=0 loops=1)
Filter: ((a <> 1) AND (a <> $1))
Planning time: 0.021 ms
Execution time: 0.068 ms
(7 rows)

[1]: /messages/by-id/5ebae4cf-8145-975c-ad75-16eb7f756f32@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v3.patchapplication/octet-stream; name=runtime_prune_drowley_v3.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index b2a2ab6..ceae604 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -279,25 +281,32 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext,List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext, bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static int32 partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1684,7 +1693,8 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
-							List *partclauses)
+							ParamListInfo prmlist, ExprContext *econtext,
+							 List *partclauses)
 {
 	Bitmapset	   *result;
 	List		   *partconstr = RelationGetPartitionQual(relation);
@@ -1717,12 +1727,128 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that's already been pruned. Set them all to
+	 * this initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths who's rel is not directly parented by rel.  We'll
+		 * process any we skip here later when looping through partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subnode.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the tables returned by get_partitions_from_clauses may be other
+	 * partitioned tables.  Unlike the case above, these won't be subnodes of
+	 * the Append.  To handle these we must make use of another
+	 * PartitionPruneInfo which we must use to recursively search until we
+	 * reach the leaf partitions.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the relation being queried.  We
+		 * only care about sub-partition parents here.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel. We'll search until we get down to the leaf partitions by
+		 * recursively calling ourself.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1734,6 +1860,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1750,8 +1878,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1784,6 +1913,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	if (ne_clauses)
 		result = bms_int_members(result,
 								 get_partitions_from_ne_clauses(relation,
+																prmlist,
+																econtext,
 																ne_clauses));
 
 	/*
@@ -1800,6 +1931,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		result = bms_int_members(result,
 								 get_partitions_from_or_clause_args(relation,
 																	rt_index,
+																	prmlist,
+																	econtext,
 																	or->args));
 	}
 
@@ -1863,7 +1996,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1898,7 +2032,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1965,6 +2100,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -1997,6 +2134,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2051,7 +2190,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2480,7 +2622,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2521,6 +2663,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			if (op_strategy < 0 &&
 				need_next_max &&
 				partkey_datum_from_expr(partkey, i, constarg,
+										prmlist, econtext,
 										&keys->maxkeys[i]))
 			{
 				keys->n_maxkeys++;
@@ -2533,11 +2676,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				Assert(incl);
 				if (need_next_eq &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->eqkeys[i]))
 					keys->n_eqkeys++;
 
 				if (need_next_max &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->maxkeys[i]))
 				{
 					keys->n_maxkeys++;
@@ -2546,6 +2691,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 				if (need_next_min &&
 					partkey_datum_from_expr(partkey, i, constarg,
+											prmlist, econtext,
 											&keys->minkeys[i]))
 				{
 					keys->n_minkeys++;
@@ -2554,6 +2700,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 			}
 			else if (need_next_min &&
 					 partkey_datum_from_expr(partkey, i, constarg,
+											 prmlist, econtext,
 											 &keys->minkeys[i]))
 			{
 				keys->n_minkeys++;
@@ -2642,7 +2789,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist, ExprContext *econtext,
+						Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2684,6 +2832,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2702,6 +2887,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2745,7 +2931,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2800,7 +2986,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2856,7 +3042,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2886,7 +3072,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2904,7 +3090,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2938,6 +3124,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -2947,10 +3134,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..c8dee20 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,6 +57,7 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
@@ -82,6 +83,9 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node,
+						   PartitionPruneInfo *pinfo);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +131,27 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
+
+	/*
+	 * Miscellaneous initialization
+	 *
+	 * create expression context for node
+	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
+	/*
+	 * The presence  of a part_prune_info means that run-time pruning is
+	 * enabled, so here we'll determine which subplans need to be scanned.
+	 */
+	if (node->part_prune_info)
+		set_valid_runtime_subplans(appendstate, appendstate->part_prune_info);
+
+	/* else, mark all subplans as requiring a scan */
+	else
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
 
 	/*
 	 * Miscellaneous initialization
@@ -163,10 +188,20 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Parallel-aware append plans must choose the first subplan to execute by
 	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
+	 * always start with the first valid subplan.
 	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (appendstate->ps.plan->parallel_aware)
+		appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		appendstate->as_whichplan = bms_next_member(
+										appendstate->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (appendstate->as_whichplan < 0)
+			appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +292,22 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * The presence of a part_prune_info means that run-time pruning is
+	 * enabled.  If any of the parameters being used for partition pruning
+	 * have changed, then we'd better redetermine which subplans we need to
+	 * scan.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		/* Free previous valid subplan bits, if any */
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+
+		set_valid_runtime_subplans(node, node->part_prune_info);
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +327,17 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (node->ps.plan->parallel_aware)
+		node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		node->as_whichplan = bms_next_member(node->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (node->as_whichplan < 0)
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +426,19 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
 	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
 	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -399,6 +456,7 @@ choose_next_subplan_for_leader(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -412,21 +470,31 @@ choose_next_subplan_for_leader(AppendState *node)
 	}
 	else
 	{
-		/* Start with last subplan. */
-		node->as_whichplan = node->as_nplans - 1;
+		/* Start with last valid subplan. */
+		node->as_whichplan = bms_prev_member(validplans, -1);
+
+		/* Bail if there are no valid plans */
+		if (node->as_whichplan < 0)
+		{
+			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+			LWLockRelease(&pstate->pa_lock);
+			return false;
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[node->as_whichplan])
 	{
-		if (node->as_whichplan == 0)
+		node->as_whichplan = bms_prev_member(validplans, node->as_whichplan);
+
+		if (node->as_whichplan < 0)
 		{
 			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
 			node->as_whichplan = INVALID_SUBPLAN_INDEX;
 			LWLockRelease(&pstate->pa_lock);
 			return false;
 		}
-		node->as_whichplan--;
 	}
 
 	/* If non-partial, immediately mark as finished. */
@@ -457,6 +525,7 @@ choose_next_subplan_for_worker(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -477,15 +546,26 @@ choose_next_subplan_for_worker(AppendState *node)
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[pstate->pa_next_plan])
 	{
-		if (pstate->pa_next_plan < node->as_nplans - 1)
+		int nextplan;
+
+		nextplan = bms_next_member(validplans, pstate->pa_next_plan);
+		if (nextplan >= 0)
 		{
-			/* Advance to next plan. */
-			pstate->pa_next_plan++;
+			/* Advance to next valid plan. */
+			pstate->pa_next_plan = nextplan;
 		}
 		else if (append->first_partial_plan < node->as_nplans)
 		{
-			/* Loop back to first partial plan. */
-			pstate->pa_next_plan = append->first_partial_plan;
+			/* Loop back to first valid partial plan. */
+			pstate->pa_next_plan = bms_next_member(validplans,
+											append->first_partial_plan - 1);
+
+			/*
+			 * Ensure we there is a valid first partial plan, if not then
+			 * we'll the next plan so that we finish the loop below.
+			 */
+			if (pstate->pa_next_plan < 0)
+				pstate->pa_next_plan = node->as_whichplan;
 		}
 		else
 		{
@@ -503,8 +583,9 @@ choose_next_subplan_for_worker(AppendState *node)
 	}
 
 	/* Pick the plan we found, and advance pa_next_plan one more time. */
-	node->as_whichplan = pstate->pa_next_plan++;
-	if (pstate->pa_next_plan >= node->as_nplans)
+	node->as_whichplan = pstate->pa_next_plan;
+	pstate->pa_next_plan = bms_next_member(validplans, pstate->pa_next_plan);
+	if (pstate->pa_next_plan < 0)
 	{
 		if (append->first_partial_plan < node->as_nplans)
 			pstate->pa_next_plan = append->first_partial_plan;
@@ -526,3 +607,51 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * get_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		'prune_qual'.  All subplans which provably cannot possibly have
+ *		matching records are eliminated and the remainder are set in the
+ *		AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node, PartitionPruneInfo *pinfo)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+	List		   *clauses = node->prune_qual;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			node->as_valid_subplans = bms_add_member(node->as_valid_subplans,
+													 pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans(node, pinfo->subpartindex[i]);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subnodes");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index ae30072..fb104bc 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * 1-bit in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 84d7171..f533cbe 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,31 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5034,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeaf8fd..ee53c72 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1711,6 +1719,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	List	   *partitioned_rels = NIL;
 	RangeTblEntry *rte;
 	double		partial_rows = -1;
+	bool		trypartitionprune = false;
 
 	/*
 	 * AppendPath generated for partitioned tables must record the RT indexes
@@ -1735,7 +1744,19 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		rte = planner_rt_fetch(rel->relid, root);
 		if (rte->rtekind == RTE_RELATION &&
 			rte->relkind == RELKIND_PARTITIONED_TABLE)
-		partitioned_rels = rel->live_partitioned_rels;
+		{
+			partitioned_rels = rel->live_partitioned_rels;
+
+			/*
+			 * For base partitioned tables we'll try to see if we can perform
+			 * any run-time partition pruning.  We need to do a bit more work
+			 * later in planning to ensure we can enable it, so this just
+			 * allows a way to save doing that work in cases when we certainly
+			 * can't enable it.
+			 */
+			if (rel->reloptkind == RELOPT_BASEREL)
+				trypartitionprune = true;
+		}
 	}
 	else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
 	{
@@ -1907,9 +1928,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
-												  partitioned_rels, -1));
+												  partitioned_rels, -1,
+												  trypartitionprune));
 
 	/*
 	 * Consider an append of unordered, unparameterized partial paths.  Make
@@ -1949,10 +1971,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
-										partitioned_rels, -1);
+										partitioned_rels, -1,
+										trypartitionprune);
 
 		/*
 		 * Make sure any subsequent partial paths use the same row count
@@ -1998,10 +2021,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
-										partitioned_rels, partial_rows);
+										partitioned_rels, partial_rows,
+										trypartitionprune);
 		add_partial_path(rel, (Path *) appendpath);
 	}
 
@@ -2054,9 +2078,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
-										partitioned_rels, -1));
+										partitioned_rels, -1,
+										trypartitionprune));
 	}
 }
 
@@ -2319,8 +2344,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..15d1426 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,8 +1232,8 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1a0d3a8..57961ec 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,53 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual = extract_actual_clauses(best_path->path.parent->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+			RangeTblEntry *rte;
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			rte = planner_rt_fetch(rel->relid, root);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1114,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,19 +5364,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ffdf9c5..6139a13 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,14 +3678,16 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
 								   0,
 								   false,
 								   NIL,
-								   -1);
+								   -1,
+								   false);
 			path->pathtarget = target;
 		}
 		else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index a24e8ac..cc08769 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,8 +590,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
 
@@ -702,8 +702,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ce83fbc..60f31e3 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 2aee156..7f3dcbb 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,11 +1211,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows)
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
 	ListCell   *l;
@@ -1224,8 +1227,20 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When using run-time partition pruning we need the clause list details
+	 * in the param_info. get_appendrel_parampathinfo does not do this, so
+	 * when we're trying to enable run-time partition pruning we'll just call
+	 * get_baserel_parampathinfo instead as it does what we need.
+	 */
+	if (trypartitionprune)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -1248,6 +1263,8 @@ create_append_path(RelOptInfo *rel,
 									  append_startup_cost_compare);
 	}
 	pathnode->first_partial_path = list_length(subpaths);
+	pathnode->part_prune_params = rel->runtime_prune_params;
+	pathnode->trypartitionprune = trypartitionprune;
 	pathnode->subpaths = list_concat(subpaths, partial_subpaths);
 
 	foreach(l, subpaths)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index dcfda1c..e673ea8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 853c1f6..e83720f 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1053,7 +1053,14 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	 * Note that if generic_cost is -1 (indicating we've not yet determined
 	 * the generic plan cost), we'll always prefer generic at this point.
 	 */
-	if (plansource->generic_cost < avg_custom_cost)
+
+	/*
+	 * XXX fix this. Currently we tend to favor custom plans when plan
+	 * time partition pruning is performed since the generic plans will
+	 * tend to cost more since they'll not have eliminated the same
+	 * partitions. For testing only, we'll just use a generic plan here.
+	 */
+	/* if (plansource->generic_cost < avg_custom_cost) */
 		return false;
 
 	return true;
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..9ff6685 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,9 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 3b62a97..b4ded9a 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 44d8c47..f3581dc 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1002,6 +1002,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1016,8 +1017,12 @@ struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Bitmapset  *as_valid_subplans;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c5b5115..b21ecfb 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index d763da6..cc57b1d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 074ae0a..19695bd 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,24 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * recursively to search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9f0b657..7d83375 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,10 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	/* ParamIds useful for subpath elimination during execution, or NULL */
+	Bitmapset  *part_prune_params;
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..4fb48b1 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index f183aac..d153aa3 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,11 +64,12 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows);
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..b98d0b9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
#48David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#46)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 01:43, Beena Emerson <memissemerson@gmail.com> wrote:

I wonder if Robert understands the extent of the problem. The patch
will be useful to prune away partitions when the partitioned table is
on the inside of a parameterised nested loop join, but I think we'll
likely get some complaints about PREPARE statements always using a
custom plan. It's going to make this pretty hard to write regression
tests for unless we went and invent some GUC "prefer_generic_plan", or
"generic_plan_cost_multiplier" to provide a way to coax
choose_custom_plan() into not choosing a custom plan. I admit to not
having the answer to this but I don't think we should rule out
discussing possible fixes.

I have attached the regression tests I am currently using to check my
patch. It has few prepared statements using runtime pruning.

Thanks for sending. Will the tests still pass if you remove the hack
from choose_custom_plan()?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#49Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#48)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Thu, Dec 21, 2017 at 6:30 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 22 December 2017 at 01:43, Beena Emerson <memissemerson@gmail.com> wrote:

I wonder if Robert understands the extent of the problem. The patch
will be useful to prune away partitions when the partitioned table is
on the inside of a parameterised nested loop join, but I think we'll
likely get some complaints about PREPARE statements always using a
custom plan. It's going to make this pretty hard to write regression
tests for unless we went and invent some GUC "prefer_generic_plan", or
"generic_plan_cost_multiplier" to provide a way to coax
choose_custom_plan() into not choosing a custom plan. I admit to not
having the answer to this but I don't think we should rule out
discussing possible fixes.

I have attached the regression tests I am currently using to check my
patch. It has few prepared statements using runtime pruning.

Thanks for sending. Will the tests still pass if you remove the hack
from choose_custom_plan()?

The hack has been removed in the v7 patch sent and yes the tests work
without any additional adjustments.

Thank you,

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#50David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#49)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 02:06, Beena Emerson <memissemerson@gmail.com> wrote:

Thanks for sending. Will the tests still pass if you remove the hack
from choose_custom_plan()?

The hack has been removed in the v7 patch sent and yes the tests work
without any additional adjustments.

Oh nice. I see you've included parameters to force the 5 custom plans
to having a higher cost, then for the first generic plan attempt,
you're choosing params that'll result in a cheaper plan. Great idea.
I'll revoke what I said about this being impossible to test now :)

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#51Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#47)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Thu, Dec 21, 2017 at 6:26 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 21 December 2017 at 22:01, David Rowley <david.rowley@2ndquadrant.com> wrote:

I've attached the latest version of the patch. This is based
on Amit's v15 of faster-partition-pruning [1] which I found to cleanly
apply to f94eec490

Well, that went out of date pretty quickly. Amit has now posted v16 of
the faster partition pruning patch [1] which conflicts with my changes
in my v2 patch.

I've attached a new version of the patch to resolve these conflicts.

Thank you for the patch, I applied it over Amit's v16 patches on
commit 180428404.

I found that make check is crashing along with the following sql you mentioned.

postgres=# create table ta (a int not null) partition by list (a);
CREATE TABLE
postgres=# create table ta1 partition of ta for values in(1,2);
CREATE TABLE
postgres=# create table ta2 partition of ta for values in(3,4);
CREATE TABLE
postgres=# explain select * from ta where a <> 1 and a <> 2;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

This seems like having two different patches for the same feature. I
will post my version of the patch which uses the struct
PartitionPruneInfo from your patch and I will add the other additional
features you added like optimizing the pruning rescan. I will try and
post the patch tomorrow.
If there is more suggestions, you can give it over that; otherwise it
seems like duplicating efforts.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#52David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#51)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 03:02, Beena Emerson <memissemerson@gmail.com> wrote:

postgres=# create table ta (a int not null) partition by list (a);
CREATE TABLE
postgres=# create table ta1 partition of ta for values in(1,2);
CREATE TABLE
postgres=# create table ta2 partition of ta for values in(3,4);
CREATE TABLE
postgres=# explain select * from ta where a <> 1 and a <> 2;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

I just get "ERROR: negative bitmapset member not allowed" here. I
reported that one to Amit over on the other thread. Not sure why
you're getting a crash. Can you get a backtrace?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#53Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#43)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Dec 21, 2017 at 4:01 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

The problem is down to the logic in choose_custom_plan() only choosing
a generic plan if the average cost of the generic plan is less than
the average custom plan cost. The problem is that the generic plan can
have many extra Append subnodes in comparison to the custom plan, all
of which are taken into account in the total plan cost, but these may
be pruned during execution. The logic in choose_custom_plan() has no
idea about this. I don't have any bright ideas on how to fix this
yet, as, suppose a PREPAREd statement like the following comes along:

PREPARE q3 (int, int) AS SELECT * FROM partitioned_table WHERE partkey
BETWEEN $1 AND $2;

the run-time pruning may prune it down no subplans, all subplans, or
any number in between. So we can't do anything like take the total
Append cost to be the highest costing of its subplans, and likely
using the average cost might not be a good idea either.

Well, I do think we need to make some kind of estimate. It may be a
bad estimate, but if we do nothing, we're estimating that no pruning
at all will happen, which is probably not right either. I mean, if we
have...

PREPARE q3 (int, int) AS SELECT * FROM unpartitioned_table WHERE
partkey BETWEEN $1 AND $2;

...that has to decide whether to use an index. And to do that it has
to estimate what fraction of the table will match the BETWEEN clause.
That may be an uninformed guess, but it guesses something. We
probably want to do something here that makes the guess for a
partitioned_table similar to the guess for an unpartitioned_table.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#54David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#51)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 03:02, Beena Emerson <memissemerson@gmail.com> wrote:

This seems like having two different patches for the same feature. I
will post my version of the patch which uses the struct
PartitionPruneInfo from your patch and I will add the other additional
features you added like optimizing the pruning rescan. I will try and
post the patch tomorrow.

I apologise for persisting in making these parallel efforts. I do have
time right now to dedicate to review this patch, but that time is
running out. At this end of this time, I was really hoping that there
would be a patch that's worthy of being committed (or at least one
worthy of a committers time). During my review of v5, because I found
the patch to still need quite a bit of work, I thought the best use of
that time was to make it work myself, which to my knowledge I have
done. Although, I'm sure my patch will still have bugs, it appears to
me to be quite a bit further ahead than your v7 WIP patch.

If there is more suggestions, you can give it over that; otherwise it
seems like duplicating efforts.

Much of the things I did differently from you could be taken as suggestions.

There were a number of things in the v7 patch were still not in a
workable state:

1. Using the PlannerInfo to record details about Append. How will this
work with a plan containing multiple Appends scanning partitioned
tables?
2. The use of AppendState->subplan_indexes List. Please use a
Bitmapset to mark the valid subplans. Lists are not particularly
efficient to get the nth item.
3. Use of PlannerInfo to store details specific to a single
partitioned table in set_base_rel_sizes.
4. Use of a new PlannerInfo->join_clauses in set_rel_size(). How will
this work when there are multiple partitioned tables being scanned in
a single plan?
5. In match_clauses_to_partkey() you're using a new
PlannerInfo->baserestrictinfo_param_indexes List to store ParamIds.
How will this work when there are multiple partitioned tables being
scanned in a single plan? A Bitmapset would be a better choice to
store paramids in.
6. In set_append_rel_pathlist you're using more PlannerInfo members to
handle a specific Append rel. Again, how will it work for plans
scanning multiple different partitioned tables?
7. Your changes to get_appendrel_parampathinfo() ignore equivalence
join clauses, don't you need to look at these too? If so, maybe it
might be worth just using get_baserel_parampathinfo()?
8. Lack of ability to detect if set_append_subplan_indexes() needs to
be called in ExecReScanAppend(). Some parameters that change might not
have an effect on which partitions to scan.

If you go and find a new way to solve all those problems, then please
consider which one of us it is that's making the duplicate effort.

Again, I'm sorry that I have been standing on your toes with my work
here. I'm certainly not out to try to take any glory here. I just want
the patch to be in a working state and the time I have to do that is
fast running out.

Please consider my efforts as an offer of assistance rather than a
threat to your work.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#55David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#53)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 12:45, Robert Haas <robertmhaas@gmail.com> wrote:

PREPARE q3 (int, int) AS SELECT * FROM unpartitioned_table WHERE
partkey BETWEEN $1 AND $2;

...that has to decide whether to use an index. And to do that it has
to estimate what fraction of the table will match the BETWEEN clause.
That may be an uninformed guess, but it guesses something. We
probably want to do something here that makes the guess for a
partitioned_table similar to the guess for an unpartitioned_table.

Are you considering some sort of clauselist_selectivity() estimate on
the given parameters and comparing that to the same selectivities that
were determined for the previous custom plans?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#56Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#55)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Dec 21, 2017 at 6:53 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 22 December 2017 at 12:45, Robert Haas <robertmhaas@gmail.com> wrote:

PREPARE q3 (int, int) AS SELECT * FROM unpartitioned_table WHERE
partkey BETWEEN $1 AND $2;

...that has to decide whether to use an index. And to do that it has
to estimate what fraction of the table will match the BETWEEN clause.
That may be an uninformed guess, but it guesses something. We
probably want to do something here that makes the guess for a
partitioned_table similar to the guess for an unpartitioned_table.

Are you considering some sort of clauselist_selectivity() estimate on
the given parameters and comparing that to the same selectivities that
were determined for the previous custom plans?

No, I don't think comparing to previous custom plans is a workable
approach. I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#57David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#56)
Re: [HACKERS] Runtime Partition Pruning

On 22 December 2017 at 14:29, Robert Haas <robertmhaas@gmail.com> wrote:

On Thu, Dec 21, 2017 at 6:53 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Are you considering some sort of clauselist_selectivity() estimate on
the given parameters and comparing that to the same selectivities that
were determined for the previous custom plans?

No, I don't think comparing to previous custom plans is a workable
approach. I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.

Okay. Do you think this is something we need to solve for this patch?
When I complained originally I didn't quite see any way to even test
the majority of this patch with the regression tests, but Beena has
since proven me wrong about that.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#58Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#57)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Dec 21, 2017 at 8:37 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

No, I don't think comparing to previous custom plans is a workable
approach. I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.

Okay. Do you think this is something we need to solve for this patch?
When I complained originally I didn't quite see any way to even test
the majority of this patch with the regression tests, but Beena has
since proven me wrong about that.

Although I have done one round of view of this patch, I haven't really
got my head around it completely yet and I haven't spent of time on it
yet, so my opinions are not as well-formed as maybe they should be.
I'm glad, by the way, that you are putting some effort into it, as I
think that will help move this forward more quickly. At a high
level, I want to avoid trying to solve too many problems in one patch
(which was the motivation behind my comment near the top of the
thread), but I also want to end up with something useful (which I
think is your concern).

Leaving aside the difficulty of implementation, I have some questions
about what the right thing to do actually is. In a simple case, I'm
guessing that the cost of creating a custom plan will exceed the
amount that the plan saves, but in more complex cases, I'm not sure
that will be true. For instance, if we know the particular parameter
value at plan time, we can make a more accurate estimate of how many
times that value appears, which can then feed into our choice of what
plan shape to use. That is, for a query like SELECT * FROM a JOIN b
ON a.x = b.x WHERE a.y = $1, the generic plan might choose, say, a
nested loop with b on the inner side, but if we know that a particular
value for $1 will match a lot of rows in a, we might prefer a hash or
merge join for that specific case. Run-time pruning doesn't give us
that flexibility. My intuition is that the more complex we make the
query, the more point there will be to making custom plans, and the
simpler the query, the more likely it is that a generic plan will be
good enough that it's not worth replanning every time.

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#59David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#47)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hi,

Please find attached my 4th version this patch.

This is now based on v17 of Amit's faster partition pruning patch [1]/messages/by-id/58c3e20a-a964-4fdb-4e7d-bd833e9bead1@lab.ntt.co.jp.
It also now includes Beena's tests which I've done some mostly
cosmetic changes to.

I've also fixed a few bugs, one in a case where I was not properly
handling zero matching partitions in nodeAppend.c.

Another change I've made is to now perform the partition pruning at
run-time using a new memory context that's reset each time we
redetermine the matching partitions. This was required since we're
calling a planner function which might not be too careful about
pfreeing memory it allocates. A test case I was running before making
this change ended out failing to palloc memory due to OOM.

I've not done anything about reducing the cost of the Append path when
runtime pruning is enabled. I'm still thinking over the best way to
handle that.

[1]: /messages/by-id/58c3e20a-a964-4fdb-4e7d-bd833e9bead1@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v4.patchapplication/octet-stream; name=runtime_prune_drowley_v4.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 9606ff5..5fdbd72 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1695,6 +1705,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1724,16 +1735,134 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that's already been pruned. Set them all to
+	 * this initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpath which belong to relations not directly parented by
+		 * rel.  We'll process any we skip below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the tables returned by get_partitions_from_clauses may be other
+	 * partitioned tables.  Unlike the case above, these won't be subpaths of
+	 * the Append.  To handle these we must create a sub-PartitionPruneInfo to
+	 * allow us to determine if subnodes which belong to sub-partitioned
+	 * tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1745,6 +1874,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1761,8 +1892,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1796,7 +1928,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1816,6 +1951,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1886,7 +2023,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1921,7 +2059,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1989,6 +2128,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2021,6 +2162,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2076,7 +2219,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2504,7 +2650,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2567,11 +2713,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2580,6 +2728,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2590,6 +2739,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2602,6 +2752,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2695,7 +2846,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist, ExprContext *econtext,
+						Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2737,6 +2889,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2755,6 +2944,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2798,7 +2988,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2853,7 +3043,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2909,7 +3099,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2939,7 +3129,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2957,7 +3147,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2991,6 +3181,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3000,10 +3191,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..8025e57 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +84,11 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,21 +134,41 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
 	/*
+	 * The presence  of a part_prune_info means that run-time pruning is
+	 * enabled, so here we'll determine which subplans need to be scanned.
+	 */
+	if (node->part_prune_info)
+	{
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+
+
+		set_valid_runtime_subplans(appendstate);
+	}
+	/* else, mark all subplans as requiring a scan */
+	else
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
 	 */
@@ -161,12 +188,22 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ps_ProjInfo = NULL;
 
 	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
+	 * Parallel-aware append plans must choose the first valid subplan to
+	 * execute by looking at shared memory, but non-parallel-aware append
+	 * plans can always start with the first valid subplan.
 	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (appendstate->ps.plan->parallel_aware)
+		appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		appendstate->as_whichplan = bms_next_member(
+										appendstate->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (appendstate->as_whichplan < 0)
+			appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +294,16 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * The presence of a part_prune_info means that run-time pruning is
+	 * enabled.  If any of the parameters being used for partition pruning
+	 * have changed, then we'd better redetermine which subplans we need to
+	 * scan.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+		set_valid_runtime_subplans(node);
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +323,17 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	if (node->ps.plan->parallel_aware)
+		node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	else
+	{
+		/* Set the plan to the first valid subplan */
+		node->as_whichplan = bms_next_member(node->as_valid_subplans, -1);
+
+		/* There mightn't be a single valid subplan. */
+		if (node->as_whichplan < 0)
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+	}
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +422,23 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
+
+	/* Handle case when all subplans were pruned */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
+		return false;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
+	/* Ensure whichplan is valid */
 	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -399,6 +456,7 @@ choose_next_subplan_for_leader(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -412,21 +470,31 @@ choose_next_subplan_for_leader(AppendState *node)
 	}
 	else
 	{
-		/* Start with last subplan. */
-		node->as_whichplan = node->as_nplans - 1;
+		/* Start with last valid subplan. */
+		node->as_whichplan = bms_prev_member(validplans, -1);
+
+		/* Bail if there are no valid plans */
+		if (node->as_whichplan < 0)
+		{
+			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
+			node->as_whichplan = INVALID_SUBPLAN_INDEX;
+			LWLockRelease(&pstate->pa_lock);
+			return false;
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[node->as_whichplan])
 	{
-		if (node->as_whichplan == 0)
+		node->as_whichplan = bms_prev_member(validplans, node->as_whichplan);
+
+		if (node->as_whichplan < 0)
 		{
 			pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
 			node->as_whichplan = INVALID_SUBPLAN_INDEX;
 			LWLockRelease(&pstate->pa_lock);
 			return false;
 		}
-		node->as_whichplan--;
 	}
 
 	/* If non-partial, immediately mark as finished. */
@@ -457,6 +525,7 @@ choose_next_subplan_for_worker(AppendState *node)
 {
 	ParallelAppendState *pstate = node->as_pstate;
 	Append	   *append = (Append *) node->ps.plan;
+	Bitmapset  *validplans = node->as_valid_subplans;
 
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
@@ -477,15 +546,26 @@ choose_next_subplan_for_worker(AppendState *node)
 	/* Loop until we find a subplan to execute. */
 	while (pstate->pa_finished[pstate->pa_next_plan])
 	{
-		if (pstate->pa_next_plan < node->as_nplans - 1)
+		int nextplan;
+
+		nextplan = bms_next_member(validplans, pstate->pa_next_plan);
+		if (nextplan >= 0)
 		{
-			/* Advance to next plan. */
-			pstate->pa_next_plan++;
+			/* Advance to next valid plan. */
+			pstate->pa_next_plan = nextplan;
 		}
 		else if (append->first_partial_plan < node->as_nplans)
 		{
-			/* Loop back to first partial plan. */
-			pstate->pa_next_plan = append->first_partial_plan;
+			/* Loop back to first valid partial plan. */
+			pstate->pa_next_plan = bms_next_member(validplans,
+											append->first_partial_plan - 1);
+
+			/*
+			 * Ensure there is a valid first partial plan, if not then
+			 * arrange to bail out.
+			 */
+			if (pstate->pa_next_plan < 0)
+				pstate->pa_next_plan = node->as_whichplan;
 		}
 		else
 		{
@@ -503,11 +583,27 @@ choose_next_subplan_for_worker(AppendState *node)
 	}
 
 	/* Pick the plan we found, and advance pa_next_plan one more time. */
-	node->as_whichplan = pstate->pa_next_plan++;
-	if (pstate->pa_next_plan >= node->as_nplans)
+	node->as_whichplan = pstate->pa_next_plan;
+	pstate->pa_next_plan = bms_next_member(validplans, pstate->pa_next_plan);
+	if (pstate->pa_next_plan < 0)
 	{
 		if (append->first_partial_plan < node->as_nplans)
-			pstate->pa_next_plan = append->first_partial_plan;
+		{
+			/*
+			 * If we have any partial plans then let the next caller can work
+			 * on the first valid partial plan.
+			 */
+			pstate->pa_next_plan = bms_next_member(validplans,
+											append->first_partial_plan - 1);
+
+			/*
+			 * If there are no valid partial plans then the next caller has
+			 * nothing to do.
+			 */
+			if (pstate->pa_next_plan < 0)
+				pstate->pa_next_plan = INVALID_SUBPLAN_INDEX;
+
+		}
 		else
 		{
 			/*
@@ -526,3 +622,84 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * get_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		AppendState's 'prune_qual'.  All subplans which provably cannot
+ *		possibly have matching records are eliminated and the remainder are
+ *		set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldContext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Free any previously set valid subplans */
+	bms_free(node->as_valid_subplans);
+
+	/*
+	 * Since we're calling s planner function to determine which partitions
+	 * are valid with the current parameters, we must switch to a temp memory
+	 * context as the planner code may not be too careful about memory
+	 * allocations.  We may call this function many times over from
+	 * ExecReScanAppend, so we certainly don't want any leaks from the
+	 * Executor's context.
+	 */
+	oldContext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldContext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+	List		   *clauses = node->prune_qual;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subnodes");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index ae30072..d382cde 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 84d7171..f533cbe 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,31 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5034,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeaf8fd..ee53c72 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1711,6 +1719,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	List	   *partitioned_rels = NIL;
 	RangeTblEntry *rte;
 	double		partial_rows = -1;
+	bool		trypartitionprune = false;
 
 	/*
 	 * AppendPath generated for partitioned tables must record the RT indexes
@@ -1735,7 +1744,19 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		rte = planner_rt_fetch(rel->relid, root);
 		if (rte->rtekind == RTE_RELATION &&
 			rte->relkind == RELKIND_PARTITIONED_TABLE)
-		partitioned_rels = rel->live_partitioned_rels;
+		{
+			partitioned_rels = rel->live_partitioned_rels;
+
+			/*
+			 * For base partitioned tables we'll try to see if we can perform
+			 * any run-time partition pruning.  We need to do a bit more work
+			 * later in planning to ensure we can enable it, so this just
+			 * allows a way to save doing that work in cases when we certainly
+			 * can't enable it.
+			 */
+			if (rel->reloptkind == RELOPT_BASEREL)
+				trypartitionprune = true;
+		}
 	}
 	else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
 	{
@@ -1907,9 +1928,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
-												  partitioned_rels, -1));
+												  partitioned_rels, -1,
+												  trypartitionprune));
 
 	/*
 	 * Consider an append of unordered, unparameterized partial paths.  Make
@@ -1949,10 +1971,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
-										partitioned_rels, -1);
+										partitioned_rels, -1,
+										trypartitionprune);
 
 		/*
 		 * Make sure any subsequent partial paths use the same row count
@@ -1998,10 +2021,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
-										partitioned_rels, partial_rows);
+										partitioned_rels, partial_rows,
+										trypartitionprune);
 		add_partial_path(rel, (Path *) appendpath);
 	}
 
@@ -2054,9 +2078,10 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
-										partitioned_rels, -1));
+										partitioned_rels, -1,
+										trypartitionprune));
 	}
 }
 
@@ -2319,8 +2344,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/*
 	 * We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..15d1426 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,8 +1232,8 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
-											  0, false, NIL, -1));
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
+											  0, false, NIL, -1, false));
 
 	/* Set or update cheapest_total_path and related fields */
 	set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1a9fd82..0c2e7fe 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,49 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual = extract_actual_clauses(best_path->path.parent->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1110,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,19 +5360,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ffdf9c5..6139a13 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,14 +3678,16 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
 								   0,
 								   false,
 								   NIL,
-								   -1);
+								   -1,
+								   false);
 			path->pathtarget = target;
 		}
 		else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f87849e..d320883 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,8 +590,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
 
@@ -702,8 +702,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
-									   NULL, 0, false, NIL, -1);
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
+									   NULL, 0, false, NIL, -1, false);
 
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 93eb374..e4c9191 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 2aee156..7f3dcbb 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,11 +1211,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows)
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune)
 {
 	AppendPath *pathnode = makeNode(AppendPath);
 	ListCell   *l;
@@ -1224,8 +1227,20 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When using run-time partition pruning we need the clause list details
+	 * in the param_info. get_appendrel_parampathinfo does not do this, so
+	 * when we're trying to enable run-time partition pruning we'll just call
+	 * get_baserel_parampathinfo instead as it does what we need.
+	 */
+	if (trypartitionprune)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -1248,6 +1263,8 @@ create_append_path(RelOptInfo *rel,
 									  append_startup_cost_compare);
 	}
 	pathnode->first_partial_path = list_length(subpaths);
+	pathnode->part_prune_params = rel->runtime_prune_params;
+	pathnode->trypartitionprune = trypartitionprune;
 	pathnode->subpaths = list_concat(subpaths, partial_subpaths);
 
 	foreach(l, subpaths)
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index dcfda1c..e673ea8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..9ff6685 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,9 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 3b62a97..b4ded9a 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index c9a5279..4467bd3 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1005,6 +1005,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1019,8 +1020,13 @@ struct AppendState
 	PlanState **appendplans;	/* array of PlanStates for my inputs */
 	int			as_nplans;
 	int			as_whichplan;
+	Bitmapset  *as_valid_subplans;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c5b5115..b21ecfb 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index d763da6..cc57b1d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 074ae0a..cdb7d61 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,24 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9f0b657..7d83375 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,10 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	/* ParamIds useful for subpath elimination during execution, or NULL */
+	Bitmapset  *part_prune_params;
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..4fb48b1 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index f183aac..d153aa3 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,11 +64,12 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
-				   List *partitioned_rels, double rows);
+				   List *partitioned_rels, double rows,
+				   bool trypartitionprune);
 extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *subpaths,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..b98d0b9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..7b86e6e 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,500 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+deallocate ab_q1;
+-- join
+drop table if exists tbl1;
+NOTICE:  table "tbl1" does not exist, skipping
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+drop table if exists tprt;
+NOTICE:  table "tprt" does not exist, skipping
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=17 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=4 loops=4)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=3)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=1 loops=4)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+(17 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+-- parallel append
+prepare ab_q1 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q1 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+deallocate ab_q1;
+drop table ab, tbl1, tprt;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..d7b359d 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,154 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+deallocate ab_q1;
+
+-- join
+drop table if exists tbl1;
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+drop table if exists tprt;
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- parallel append
+prepare ab_q1 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+
+deallocate ab_q1;
+
+drop table ab, tbl1, tprt;
#60Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#59)
Re: [HACKERS] Runtime Partition Pruning

Hello David,

On Wed, Dec 27, 2017 at 8:36 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Hi,

Please find attached my 4th version this patch.

Thanks for the patch

This is now based on v17 of Amit's faster partition pruning patch [1].
It also now includes Beena's tests which I've done some mostly
cosmetic changes to.

I've also fixed a few bugs, one in a case where I was not properly
handling zero matching partitions in nodeAppend.c.

Another change I've made is to now perform the partition pruning at
run-time using a new memory context that's reset each time we
redetermine the matching partitions. This was required since we're
calling a planner function which might not be too careful about
pfreeing memory it allocates. A test case I was running before making
this change ended out failing to palloc memory due to OOM.

I've not done anything about reducing the cost of the Append path when
runtime pruning is enabled. I'm still thinking over the best way to
handle that.

I think you are testing without asserts

The following assert fails: src/backend/optimizer/plan/setrefs.c :
set_plan_refs: ln 921
Assert(splan->plan.qual == NIL);
Append node now has runtime partition quals.

Also since the valid subplans are set in ExecInitAppend, the queries
with Init Plans do not work. I had moved it to ExecAppend in my patch
to handle the InitPlans as well.

DROP TABLE IF EXISTS prun_test_part;
CREATE TABLE prun_test_part (sal int) PARTITION BY RANGE(sal);
CREATE TABLE prun_test_part_p1 PARTITION OF prun_test_part FOR VALUES
FROM (0) TO (100);
CREATE TABLE prun_test_part_p2 PARTITION OF prun_test_part FOR VALUES
FROM (100) TO (200);
CREATE TABLE prun_test_part_p3 PARTITION OF prun_test_part FOR VALUES
FROM (200) TO (300);
CREATE TABLE prun_test_part_p4 PARTITION OF prun_test_part FOR VALUES
FROM (300) TO (400);

INSERT INTO prun_test_part VALUES (90), (100), (110), (200), (210),
(300), (310);
=# explain (analyze, costs off, summary off, timing off) SELECT * FROM
prun_test_part WHERE sal < (SELECT sal FROM prun_test_part WHERE sal =
200);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#61David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#60)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 1 January 2018 at 19:22, Beena Emerson <memissemerson@gmail.com> wrote:

I think you are testing without asserts

Yeah, I was indeed. Oops.

The following assert fails: src/backend/optimizer/plan/setrefs.c :
set_plan_refs: ln 921
Assert(splan->plan.qual == NIL);
Append node now has runtime partition quals.

Also since the valid subplans are set in ExecInitAppend, the queries
with Init Plans do not work. I had moved it to ExecAppend in my patch
to handle the InitPlans as well.

Thanks for noticing. I've now changed things around so this case works
as it should and I've added a test too.

I've attached an updated patch which also fixes a number of other
problems with my previous patch.

1. The Bitmapset I was using in nodeAppend.c to mark the valid
subplans was pretty bogus for Parallel Append since the memory for the
set was not in shared memory. I changed things around to reuse the
pa_finished[] array and the patch just now sets pa_finished to true
for any invalid subplans.
2. I've added a new memory context to use in nodeAppend.c which is
used to call the planner code to determine which partitions are valid.
I'd been trying to have Amit be careful to pfree() everything in his
v17 patch, but I realised it was just not possible to get everything
pfree'd. I found it pretty easy to construct a test case which caused
an OOM.
3. I've added support for IN lists to be pruned when the IN() list
contains a parameter. The changes I made to support this case probably
mostly belong in Amit's faster partition pruning patch, but I've put
them here for now to get this case working. There's a bunch of new
tests to test this.
4. Various other cosmetic improvements.

The attached patch should be applied after patching master with Amit's
v17 faster partition pruning patch [1]/messages/by-id/58c3e20a-a964-4fdb-4e7d-bd833e9bead1@lab.ntt.co.jp.

[1]: /messages/by-id/58c3e20a-a964-4fdb-4e7d-bd833e9bead1@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v5.patchapplication/octet-stream; name=runtime_prune_drowley_v5.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 9606ff5..65d8d93 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1695,6 +1705,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1724,16 +1735,134 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1745,6 +1874,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1761,8 +1892,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1796,7 +1928,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1816,6 +1951,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1886,7 +2023,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1921,7 +2059,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1989,6 +2128,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2021,6 +2162,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2076,7 +2219,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2381,10 +2527,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				elem_clauses = NIL;
 				foreach(lc1, elem_exprs)
 				{
-					Const  *rightop = castNode(Const, lfirst(lc1));
+					Expr   *rightop = (Expr *) lfirst(lc1);
 					Expr   *elem_clause;
 
-					if (rightop->constisnull)
+					if (IsA(rightop, Const) && ((Const *)rightop)->constisnull)
 					{
 						NullTest *nulltest = makeNode(NullTest);
 
@@ -2504,7 +2650,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2567,11 +2713,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2580,6 +2728,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2590,6 +2739,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2602,6 +2752,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2695,7 +2846,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2737,6 +2889,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2755,6 +2944,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2798,7 +2988,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2853,7 +3043,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2909,7 +3099,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2939,7 +3129,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2957,7 +3147,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2991,6 +3181,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3000,10 +3191,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 0e93713..36a5a72 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +84,12 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,20 +135,47 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid now.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -160,13 +195,8 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +287,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +318,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +408,34 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle by returning
+	 * a negative number which will allow us to exit returning a false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -414,6 +468,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +532,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -526,3 +602,115 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		AppendState's 'prune_qual'.  All subplans which provably cannot
+ *		possibly have matching records are eliminated and the remainder are
+ *		set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+	List		   *clauses = node->prune_qual;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/*
+	 * First mark all plans as finished.  XXX maybe it's better to create an
+	 * inverse bitmap and just set those as finished?
+	 */
+	memset(node->as_pstate->pa_finished, 1, sizeof(bool) * node->as_nplans);
+
+	/* Now mark all the valid subplans as not finished */
+	i = -1;
+	while ((i = bms_next_member(node->as_valid_subplans, i)) >= 0)
+		node->as_pstate->pa_finished[i] = false;
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index ae30072..d382cde 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 84d7171..f533cbe 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,31 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5034,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeaf8fd..5c0fa8b 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1187,10 +1195,32 @@ match_clauses_to_partkey(PlannerInfo *root,
 
 				/* OK to add to the result. */
 				result = lappend(result, clause);
-				if (IsA(estimate_expression_value(root, rightop), Const))
+
+				if (IsA(rightop, Const))
 					*contains_const = true;
-				else
-					*contains_const = false;
+				else if (IsA(rightop, ArrayExpr))
+				{
+					ArrayExpr *arrayexpr = (ArrayExpr *) rightop;
+					ListCell   *lc;
+					bool		allconsts = true;
+
+					foreach(lc, arrayexpr->elements)
+					{
+						Expr *expr = (Expr *) lfirst(lc);
+
+						if (IsA(expr, Const))
+							continue;
+
+						allconsts = false;
+
+						if (IsA(expr, Param))
+							*paramids = bms_add_member(*paramids,
+												   ((Param *) expr)->paramid);
+					}
+
+					if (allconsts)
+						*contains_const = true;
+				}
 			}
 			else if (IsA(clause, NullTest))
 			{
@@ -1907,7 +1937,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1949,8 +1979,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1998,7 +2028,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2054,7 +2084,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2319,7 +2349,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 5bd3031..26a5f6e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1a9fd82..cb232cf 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,51 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1112,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,19 +5362,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ffdf9c5..bc7c56d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index b5c4124..0cfe010 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -915,10 +915,10 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 
 				/*
 				 * Append, like Sort et al, doesn't actually evaluate its
-				 * targetlist or check quals.
+				 * targetlist or check quals. However, quals may be used
+				 * to allow partitions to be pruned at execution time.
 				 */
 				set_dummy_tlist_references(plan, rtoffset);
-				Assert(splan->plan.qual == NIL);
 				foreach(l, splan->partitioned_rels)
 				{
 					lfirst_int(l) += rtoffset;
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f87849e..2379a0f 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 93eb374..e4c9191 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2470,6 +2470,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 2aee156..d1b78f6 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,7 +1211,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1226,37 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths, doing anything extra in this case would be wasted
+	 * work.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index dcfda1c..e673ea8 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 7a5ab45..9ff6685 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,9 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 3b62a97..b4ded9a 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 94351ea..bce9d69 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1008,6 +1008,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1024,6 +1025,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c5b5115..b21ecfb 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index d763da6..cc57b1d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 074ae0a..cdb7d61 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,24 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9f0b657..28cd543 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 1ef13a4..4fb48b1 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index f183aac..c553807 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index ea886b6..b98d0b9 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..5305cff 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,903 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and one value that we should find.
+insert into lprt_a values(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=101)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=101)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=1)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=101 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=101 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=17 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=4 loops=4)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=3)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 8
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=4 loops=1)
+   ->  Append (actual rows=1 loops=4)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+(17 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..0bc831d 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,224 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and one value that we should find.
+insert into lprt_a values(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
#62Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#61)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

I tried this patch (applying it on Amit's last current version on top of
4e2970f8807f which is the latest it applies to) and regression tests
fail with the attached diff; in all cases it appears to be an off-by-one
in row count. Would you please give it a look?

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

regression.diffstext/plain; charset=us-asciiDownload
*** /pgsql/source/master/src/test/regress/expected/partition_prune.out	2018-01-04 13:29:04.518589165 -0300
--- /home/alvherre/Code/pgsql/build/master/src/test/regress/results/partition_prune.out	2018-01-04 13:33:41.003865074 -0300
***************
*** 1836,1842 ****
           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=2)
                             Filter: (a = ANY ('{0,0,1}'::integer[]))
                       ->  Append (actual rows=0 loops=101)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
--- 1836,1842 ----
           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=50 loops=2)
                             Filter: (a = ANY ('{0,0,1}'::integer[]))
                       ->  Append (actual rows=0 loops=101)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
***************
*** 1919,1927 ****
           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=2)
                             Filter: (a = ANY ('{1,0,0}'::integer[]))
!                            Rows Removed by Filter: 1
                       ->  Append (actual rows=0 loops=101)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
                                   Index Cond: (a = a.a)
--- 1919,1927 ----
           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=50 loops=2)
                             Filter: (a = ANY ('{1,0,0}'::integer[]))
!                            Rows Removed by Filter: 0
                       ->  Append (actual rows=0 loops=101)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=1)
                                   Index Cond: (a = a.a)
***************
*** 1964,1970 ****
                 ->  Nested Loop (actual rows=0 loops=2)
                       ->  Parallel Seq Scan on lprt_a a (actual rows=50 loops=2)
                             Filter: (a = ANY ('{1,0,0}'::integer[]))
!                            Rows Removed by Filter: 1
                       ->  Append (actual rows=0 loops=100)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
                                   Index Cond: (a = a.a)
--- 1964,1970 ----
                 ->  Nested Loop (actual rows=0 loops=2)
                       ->  Parallel Seq Scan on lprt_a a (actual rows=50 loops=2)
                             Filter: (a = ANY ('{1,0,0}'::integer[]))
!                            Rows Removed by Filter: 0
                       ->  Append (actual rows=0 loops=100)
                             ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
                                   Index Cond: (a = a.a)
***************
*** 2203,2209 ****
           ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
                 Index Cond: (col1 = tbl1.col1)
                 Heap Fetches: 2
!          ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
                 Index Cond: (col1 = tbl1.col1)
                 Heap Fetches: 1
           ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
--- 2203,2209 ----
           ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
                 Index Cond: (col1 = tbl1.col1)
                 Heap Fetches: 2
!          ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=2)
                 Index Cond: (col1 = tbl1.col1)
                 Heap Fetches: 1
           ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)

======================================================================

#63David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#62)
Re: [HACKERS] Runtime Partition Pruning

On 5 January 2018 at 05:37, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I tried this patch (applying it on Amit's last current version on top of
4e2970f8807f which is the latest it applies to) and regression tests
fail with the attached diff; in all cases it appears to be an off-by-one
in row count. Would you please give it a look?

Looks like it's down to ExplainPropertyFloat() having
machine-dependent behaviour.

On the machine that I was working with when testing this the following
code outputs "1"

#include <stdio.h>

double nfiltered = 1.0;
double nloops = 2.0;

int main(void)
{
printf("%.*f", 0, nfiltered / nloops);
return 0;
}

but on your machine it must be outputting "0"?

I'm not sure if I should just try to have even row numbers in the
tests, or if we should be fixing EXPLAIN so it is consistent.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#64David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#62)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 5 January 2018 at 05:37, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I tried this patch (applying it on Amit's last current version on top of
4e2970f8807f which is the latest it applies to) and regression tests
fail with the attached diff; in all cases it appears to be an off-by-one
in row count. Would you please give it a look?

Thanks for testing. I've attached an updated patch which hopefully fixes this.

I've only thing I did to fix it was to alter the tests a bit so that
the row counts in explain are evenly divisible by the nloops or
parallel workers. Looks like it was failing due to platform dependent
behaviour in printf.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v5.patchapplication/octet-stream; name=runtime_prune_drowley_v5.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index ab60410..eb63ce8 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1695,6 +1705,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1724,16 +1735,134 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1745,6 +1874,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1761,8 +1892,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1796,7 +1928,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1816,6 +1951,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1886,7 +2023,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1921,7 +2059,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1989,6 +2128,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2021,6 +2162,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2076,7 +2219,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2381,10 +2527,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				elem_clauses = NIL;
 				foreach(lc1, elem_exprs)
 				{
-					Const  *rightop = castNode(Const, lfirst(lc1));
+					Expr   *rightop = (Expr *) lfirst(lc1);
 					Expr   *elem_clause;
 
-					if (rightop->constisnull)
+					if (IsA(rightop, Const) && ((Const *)rightop)->constisnull)
 					{
 						NullTest *nulltest = makeNode(NullTest);
 
@@ -2504,7 +2650,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2567,11 +2713,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2580,6 +2728,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2590,6 +2739,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2602,6 +2752,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2695,7 +2846,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2737,6 +2889,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2755,6 +2944,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2798,7 +2988,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2853,7 +3043,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2909,7 +3099,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2939,7 +3129,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2957,7 +3147,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2991,6 +3181,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3000,10 +3191,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 64a17fb..9eaf9c7 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,11 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +84,12 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,20 +135,47 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid now.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -160,13 +195,8 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +287,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +318,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +408,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -414,6 +469,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -466,6 +532,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +602,111 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		AppendState's 'prune_qual'.  All subplans which provably cannot
+ *		possibly have matching records are eliminated and the remainder are
+ *		set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+	List		   *clauses = node->prune_qual;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index 733fe3c..6afc90e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ddbbc79..5784540 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,31 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5034,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6774e08..e52dcb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1187,10 +1195,32 @@ match_clauses_to_partkey(PlannerInfo *root,
 
 				/* OK to add to the result. */
 				result = lappend(result, clause);
-				if (IsA(estimate_expression_value(root, rightop), Const))
+
+				if (IsA(rightop, Const))
 					*contains_const = true;
-				else
-					*contains_const = false;
+				else if (IsA(rightop, ArrayExpr))
+				{
+					ArrayExpr *arrayexpr = (ArrayExpr *) rightop;
+					ListCell   *lc;
+					bool		allconsts = true;
+
+					foreach(lc, arrayexpr->elements)
+					{
+						Expr *expr = (Expr *) lfirst(lc);
+
+						if (IsA(expr, Const))
+							continue;
+
+						allconsts = false;
+
+						if (IsA(expr, Param))
+							*paramids = bms_add_member(*paramids,
+												   ((Param *) expr)->paramid);
+					}
+
+					if (allconsts)
+						*contains_const = true;
+				}
 			}
 			else if (IsA(clause, NullTest))
 			{
@@ -1907,7 +1937,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1949,8 +1979,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1998,7 +2028,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2054,7 +2084,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2319,7 +2349,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d9249f4..1bd92c1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e599283..cd51f77 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,51 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1112,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,19 +5362,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b0f6051..0eff96a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 4617d12..2709ae2 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -915,10 +915,10 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 
 				/*
 				 * Append, like Sort et al, doesn't actually evaluate its
-				 * targetlist or check quals.
+				 * targetlist or check quals. However, quals may be used
+				 * to allow partitions to be pruned at execution time.
 				 */
 				set_dummy_tlist_references(plan, rtoffset);
-				Assert(splan->plan.qual == NIL);
 				foreach(l, splan->partitioned_rels)
 				{
 					lfirst_int(l) += rtoffset;
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 5a08e75..64a981e 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ccfae4f..8322665 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 7df8761..b524c34 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,7 +1211,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1226,37 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths, doing anything extra in this case would be wasted
+	 * work.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4b5d50e..e98f4d7 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 294e87f..2368c91 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -73,5 +74,9 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 15397e9..951ddb8 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 3ad58cd..41daa57 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1007,6 +1007,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1023,6 +1024,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 2eb3d6d..9788c35 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 74e9fb5..1d57748 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..2813146 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,24 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 855d51e..0f0bc52 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9b4288a..44f34a9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 0072b7a..8175c3f 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..2659589 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,909 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..21d6c9b 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,224 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
#65Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#63)
Re: [HACKERS] Runtime Partition Pruning

David Rowley wrote:

Looks like it's down to ExplainPropertyFloat() having
machine-dependent behaviour.

On the machine that I was working with when testing this the following
code outputs "1"
[ sample code ]

but on your machine it must be outputting "0"?

Yeah, it does. Thanks for updating --- I'll look at your patch
tomorrow.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#66Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#64)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Fri, Jan 5, 2018 at 6:24 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 5 January 2018 at 05:37, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I tried this patch (applying it on Amit's last current version on top of
4e2970f8807f which is the latest it applies to) and regression tests
fail with the attached diff; in all cases it appears to be an off-by-one
in row count. Would you please give it a look?

Thanks for testing. I've attached an updated patch which hopefully fixes this.

I've only thing I did to fix it was to alter the tests a bit so that
the row counts in explain are evenly divisible by the nloops or
parallel workers. Looks like it was failing due to platform dependent
behaviour in printf.

It does not handle change in column order (varattno) in subpartitions.

In the following case a2 has different column order
drop table ab_c;
create table ab_c (a int not null, b int) partition by list(a);

--a2 with different col order
create table abc_a2 (b int, a int not null) partition by list(b);
create table abc_a2_b1 partition of abc_a2 for values in (1);
create table abc_a2_b2 partition of abc_a2 for values in (2);
create table abc_a2_b3 partition of abc_a2 for values in (3);
alter table ab_c attach partition abc_a2 for values in (2);

--a1 and a3 with same col order as the parent
create table abc_a1 partition of ab_c for values in(1) partition by list (b);
create table abc_a1_b1 partition of abc_a1 for values in (1);
create table abc_a1_b2 partition of abc_a1 for values in (2);
create table abc_a1_b3 partition of abc_a1 for values in (3);
create table abc_a3 partition of ab_c for values in(3) partition by list (b);
create table abc_a3_b1 partition of abc_a3 for values in (1);
create table abc_a3_b2 partition of abc_a3 for values in (2);
create table abc_a3_b3 partition of abc_a3 for values in (3);

deallocate abc_q1;
prepare abc_q1 (int, int, int) as select * from ab_c where a BETWEEN
$1 and $2 AND b <= $3;

--optimizer pruning
explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 3, 1);
QUERY PLAN
------------------------------------------------------
Append (actual rows=0 loops=1)
-> Seq Scan on abc_a1_b1 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 3) AND (b <= 1))
-> Seq Scan on abc_a2_b1 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 3) AND (b <= 1))
-> Seq Scan on abc_a3_b1 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 3) AND (b <= 1))
(7 rows)

--runtime pruning after 5 runs
explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 3, 1);
QUERY PLAN
---------------------------------------------------------
Append (actual rows=0 loops=1)
-> Seq Scan on abc_a1_b1 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a1_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a1_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b1 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b2 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b3 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a3_b1 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a3_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a3_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
(19 rows)

As seen partition a2 does not prune like in other 2 subpartitions - a1 and a3.
--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#67David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#66)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 6 January 2018 at 07:31, Beena Emerson <memissemerson@gmail.com> wrote:

It does not handle change in column order (varattno) in subpartitions.

Thanks for testing and finding that. I completely overlooked applying
translation of the prune qual so that it's compatible with the
sub-partition.

I've fixed this in the attached, but I did so by calling
adjust_appendrel_attrs() from the nodeAppend.c, which did, of course,
mean that the AppendRelInfo needed to be given to the executor. I was
also a bit unsure what exactly I should be doing in primnodes.h, since
I've put PartitionPruneInfo in there, but AppendRelInfo is not. I
stuck a quick declaration of AppendRelInfo in primnode.h with an XXX
comment so we don't forget to think about that again.

In all honesty, this calling planner code from the executor seems like
quite new ground for PostgreSQL, so I'm really not sure if we're
breaking any major rules or not with we've got now. Perhaps the saving
grace here is that we're not teaching the executor how to do anything
smart with these data structures, they're just given to it to pass
back to the planner function at the appropriate moment.

I've also borrowed and simplified your test case to ensure this fix
remains working. I added another level of partitioning with another
partition that has the columns in a different order again. This is to
ensure the translation code translates from the quals of the previous
level up, not the top-level. That's required since the AppendRelInfo
is only translating 1 level at a time.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v6.patchapplication/octet-stream; name=runtime_prune_drowley_v6.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 8066237..36de96e 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1698,6 +1708,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1727,16 +1738,137 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 AppendRelInfo *subpartappinfo)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->appinfo = subpartappinfo;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																	appinfo);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1748,6 +1880,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1764,8 +1898,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1799,7 +1934,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1819,6 +1957,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1889,7 +2029,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1924,7 +2065,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1992,6 +2134,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2024,6 +2168,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2079,7 +2225,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2384,10 +2533,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				elem_clauses = NIL;
 				foreach(lc1, elem_exprs)
 				{
-					Const  *rightop = castNode(Const, lfirst(lc1));
+					Expr   *rightop = (Expr *) lfirst(lc1);
 					Expr   *elem_clause;
 
-					if (rightop->constisnull)
+					if (IsA(rightop, Const) && ((Const *)rightop)->constisnull)
 					{
 						NullTest *nulltest = makeNode(NullTest);
 
@@ -2507,7 +2656,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2570,11 +2719,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2583,6 +2734,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2593,6 +2745,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2605,6 +2758,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2698,7 +2852,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2740,6 +2895,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2758,6 +2950,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2801,7 +2994,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2856,7 +3049,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2912,7 +3105,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2942,7 +3135,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2960,7 +3153,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2994,6 +3187,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3003,10 +3197,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 64a17fb..57da038 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -57,9 +57,12 @@
 
 #include "postgres.h"
 
+#include "parser/parsetree.h"
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "optimizer/prep.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +85,13 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   List *clauses,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,20 +137,47 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->prune_qual = node->plan.qual;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid now.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -160,13 +197,8 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +289,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +320,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +410,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -414,6 +471,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -466,6 +534,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +604,124 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		AppendState's 'prune_qual'.  All subplans which provably cannot
+ *		possibly have matching records are eliminated and the remainder are
+ *		set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   node->prune_qual,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   List *clauses,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+
+	/*
+	 * The presence of an AppendRelInfo here means we're dealing with a
+	 * sub-partitioned table.  This table may have columns defined in a
+	 * different order to its parent.  We'll need to translate the attribute
+	 * numbers of Vars in the clause list so that it's compatible with the
+	 * sub-partition.
+	 */
+	if (pinfo->appinfo)
+		clauses = (List *) adjust_appendrel_attrs(NULL, (Node *) clauses,
+												  1, &pinfo->appinfo);
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext, clauses);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   clauses,
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index 733fe3c..6afc90e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ddbbc79..4b2f3eb 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,32 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(appinfo);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5035,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6774e08..e52dcb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1187,10 +1195,32 @@ match_clauses_to_partkey(PlannerInfo *root,
 
 				/* OK to add to the result. */
 				result = lappend(result, clause);
-				if (IsA(estimate_expression_value(root, rightop), Const))
+
+				if (IsA(rightop, Const))
 					*contains_const = true;
-				else
-					*contains_const = false;
+				else if (IsA(rightop, ArrayExpr))
+				{
+					ArrayExpr *arrayexpr = (ArrayExpr *) rightop;
+					ListCell   *lc;
+					bool		allconsts = true;
+
+					foreach(lc, arrayexpr->elements)
+					{
+						Expr *expr = (Expr *) lfirst(lc);
+
+						if (IsA(expr, Const))
+							continue;
+
+						allconsts = false;
+
+						if (IsA(expr, Param))
+							*paramids = bms_add_member(*paramids,
+												   ((Param *) expr)->paramid);
+					}
+
+					if (allconsts)
+						*contains_const = true;
+				}
 			}
 			else if (IsA(clause, NullTest))
 			{
@@ -1907,7 +1937,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1949,8 +1979,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1998,7 +2028,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2054,7 +2084,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2319,7 +2349,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d9249f4..1bd92c1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e599283..7f78e22 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,10 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1019,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	List	   *qual = NIL;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1059,51 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		qual =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			qual = list_concat(qual, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, NULL);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1112,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params,
+					   qual);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,19 +5362,23 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams,
+			List *qual)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
 
 	plan->targetlist = tlist;
-	plan->qual = NIL;
+	plan->qual = qual;
 	plan->lefttree = NULL;
 	plan->righttree = NULL;
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b0f6051..0eff96a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 4617d12..2709ae2 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -915,10 +915,10 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 
 				/*
 				 * Append, like Sort et al, doesn't actually evaluate its
-				 * targetlist or check quals.
+				 * targetlist or check quals. However, quals may be used
+				 * to allow partitions to be pruned at execution time.
 				 */
 				set_dummy_tlist_references(plan, rtoffset);
-				Assert(splan->plan.qual == NIL);
 				foreach(l, splan->partitioned_rels)
 				{
 					lfirst_int(l) += rtoffset;
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 5a08e75..64a981e 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ccfae4f..8322665 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 7df8761..b524c34 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -17,6 +17,7 @@
 #include <math.h>
 
 #include "miscadmin.h"
+#include "catalog/partition.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/extensible.h"
 #include "optimizer/clauses.h"
@@ -1210,7 +1211,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1226,37 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths, doing anything extra in this case would be wasted
+	 * work.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4b5d50e..e98f4d7 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8423c6e..56e9883eb 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -75,5 +76,10 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 AppendRelInfo *subpartappinfo);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 15397e9..951ddb8 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2a4f740..7fbd6e9 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1000,6 +1000,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1016,6 +1017,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	List	   *prune_qual;		/* quals used for partition pruning */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 2eb3d6d..9788c35 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 74e9fb5..1d57748 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..96f2126 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,29 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+struct AppendRelInfo; /* XXX Okay? Or move the whole thing here? */
+
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	struct AppendRelInfo *appinfo; /* Used to translate the partition prune qual's
+							 * Vars to the sub-partitions Vars, or NULL if no
+							 * translation is needed. */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 855d51e..0f0bc52 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9b4288a..44f34a9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 0072b7a..8175c3f 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..edb3168 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,959 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..81ee321 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,255 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
#68David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#67)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 7 January 2018 at 00:03, David Rowley <david.rowley@2ndquadrant.com> wrote:

I've fixed this in the attached, but I did so by calling
adjust_appendrel_attrs() from the nodeAppend.c, which did, of course,
mean that the AppendRelInfo needed to be given to the executor. I was
also a bit unsure what exactly I should be doing in primnodes.h, since
I've put PartitionPruneInfo in there, but AppendRelInfo is not. I
stuck a quick declaration of AppendRelInfo in primnode.h with an XXX
comment so we don't forget to think about that again.

Actually, this was not a very smart fix for the problem. It seems much
better to make the prune qual part of PartitionPruneInfo and just have
the planner translate the qual to what's required for the partition
that the PartitionPruneInfo belongs to. This means we no longer need
to use the Append's qual to store the prune qual and that all the
pruning information for one partition is now neatly in a single
struct.

I've attached a patch which does things like this.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v7.patchapplication/octet-stream; name=runtime_prune_drowley_v7.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 8066237..5c98118 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1698,6 +1708,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1727,16 +1738,143 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			prunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																	prunequal);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1748,6 +1886,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1764,8 +1904,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1799,7 +1940,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1819,6 +1963,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1889,7 +2035,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1924,7 +2071,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1992,6 +2140,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2024,6 +2174,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2079,7 +2231,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2384,10 +2539,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				elem_clauses = NIL;
 				foreach(lc1, elem_exprs)
 				{
-					Const  *rightop = castNode(Const, lfirst(lc1));
+					Expr   *rightop = (Expr *) lfirst(lc1);
 					Expr   *elem_clause;
 
-					if (rightop->constisnull)
+					if (IsA(rightop, Const) && ((Const *) rightop)->constisnull)
 					{
 						NullTest *nulltest = makeNode(NullTest);
 
@@ -2507,7 +2662,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2570,11 +2725,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2583,6 +2740,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2593,6 +2751,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2605,6 +2764,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2698,7 +2858,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2740,6 +2901,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2758,6 +2956,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2801,7 +3000,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2856,7 +3055,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2912,7 +3111,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2942,7 +3141,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2960,7 +3159,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2994,6 +3193,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3003,10 +3203,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 64a17fb..9a0f49d 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,7 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +83,12 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,20 +134,46 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid now.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -160,13 +193,8 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +285,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +316,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +406,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -414,6 +467,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -466,6 +530,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +600,111 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext,
+										  pinfo->prunequal);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index 733fe3c..6afc90e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ddbbc79..be60bba 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,32 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5035,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6774e08..e52dcb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1187,10 +1195,32 @@ match_clauses_to_partkey(PlannerInfo *root,
 
 				/* OK to add to the result. */
 				result = lappend(result, clause);
-				if (IsA(estimate_expression_value(root, rightop), Const))
+
+				if (IsA(rightop, Const))
 					*contains_const = true;
-				else
-					*contains_const = false;
+				else if (IsA(rightop, ArrayExpr))
+				{
+					ArrayExpr *arrayexpr = (ArrayExpr *) rightop;
+					ListCell   *lc;
+					bool		allconsts = true;
+
+					foreach(lc, arrayexpr->elements)
+					{
+						Expr *expr = (Expr *) lfirst(lc);
+
+						if (IsA(expr, Const))
+							continue;
+
+						allconsts = false;
+
+						if (IsA(expr, Param))
+							*paramids = bms_add_member(*paramids,
+												   ((Param *) expr)->paramid);
+					}
+
+					if (allconsts)
+						*contains_const = true;
+				}
 			}
 			else if (IsA(clause, NullTest))
 			{
@@ -1907,7 +1937,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1949,8 +1979,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1998,7 +2028,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2054,7 +2084,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2319,7 +2349,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d9249f4..1bd92c1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e599283..5b52480 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,9 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1018,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1057,53 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1112,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,7 +5361,9 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5320,7 +5375,8 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b0f6051..0eff96a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 5a08e75..64a981e 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ccfae4f..8322665 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 7df8761..43d1fa3 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,37 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths, doing anything extra in this case would be wasted
+	 * work.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4b5d50e..e98f4d7 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8423c6e..cb49193 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -75,5 +76,10 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 15397e9..951ddb8 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2a4f740..f984769 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1000,6 +1000,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1016,6 +1017,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 2eb3d6d..9788c35 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 74e9fb5..1d57748 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..915f675 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,25 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	List	   *prunequal; /* qual list for pruning partitions */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 855d51e..0f0bc52 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9b4288a..44f34a9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 0072b7a..8175c3f 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..edb3168 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,959 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..81ee321 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,255 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
#69Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#68)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Sun, Jan 7, 2018 at 5:31 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 7 January 2018 at 00:03, David Rowley <david.rowley@2ndquadrant.com> wrote:

I've fixed this in the attached, but I did so by calling
adjust_appendrel_attrs() from the nodeAppend.c, which did, of course,
mean that the AppendRelInfo needed to be given to the executor. I was
also a bit unsure what exactly I should be doing in primnodes.h, since
I've put PartitionPruneInfo in there, but AppendRelInfo is not. I
stuck a quick declaration of AppendRelInfo in primnode.h with an XXX
comment so we don't forget to think about that again.

Actually, this was not a very smart fix for the problem. It seems much
better to make the prune qual part of PartitionPruneInfo and just have
the planner translate the qual to what's required for the partition
that the PartitionPruneInfo belongs to. This means we no longer need
to use the Append's qual to store the prune qual and that all the
pruning information for one partition is now neatly in a single
struct.

I've attached a patch which does things like this.

The pruning does not work well with char type:

Case: A subpartition has a different col order and the subpartitioned
col is type char.

drop table ab_c;
create table ab_c (a int not null, b char) partition by list(a);
create table abc_a2 (b char, a int not null) partition by list(b);
create table abc_a2_b1 partition of abc_a2 for values in ('1');
create table abc_a2_b2 partition of abc_a2 for values in ('2');
create table abc_a2_b3 partition of abc_a2 for values in ('3');
alter table ab_c attach partition abc_a2 for values in (2);
create table abc_a1 partition of ab_c for values in(1) partition by list (b);
create table abc_a1_b1 partition of abc_a1 for values in ('1');
create table abc_a1_b2 partition of abc_a1 for values in ('2');
create table abc_a1_b3 partition of abc_a1 for values in ('3');
create table abc_a3 partition of ab_c for values in(3) partition by list (b);
create table abc_a3_b1 partition of abc_a3 for values in ('1');
create table abc_a3_b2 partition of abc_a3 for values in ('2');
create table abc_a3_b3 partition of abc_a3 for values in ('3');
deallocate abc_q1;

INSERT INTO ab_c VALUES (1,'1'), (1,'2'), (1,'3');
INSERT INTO ab_c VALUES (2,'1'), (2,'2'), (2,'3');
INSERT INTO ab_c VALUES (3,'1'), (3,'2'), (3,'3');

prepare abc_q1 (int, int, char) as select * from ab_c where a BETWEEN
$1 and $2 AND b <= $3;

--after 5 runs: abc_a2_b3 is not pruned.

# explain (analyze, costs off, summary off, timing off) execute abc_q1
(1, 2, '2');
QUERY PLAN
---------------------------------------------------------
Append (actual rows=4 loops=1)
-> Seq Scan on abc_a1_b1 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a1_b2 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a1_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b1 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b2 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a2_b3 (actual rows=0 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
Rows Removed by Filter: 1
-> Seq Scan on abc_a3_b1 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a3_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
-> Seq Scan on abc_a3_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
(20 rows)

Case 2: Case with optimizer pruning
drop table ab_c;
create table ab_c (a int not null, b int) partition by list(a);
create table abc_a2 (b int, a int not null) partition by list(b);
create table abc_a2_b1 partition of abc_a2 for values in (1);
create table abc_a2_b2 partition of abc_a2 for values in (2);
create table abc_a2_b3 partition of abc_a2 for values in (3);
alter table ab_c attach partition abc_a2 for values in (2);
create table abc_a1 partition of ab_c for values in(1) partition by list (b);
create table abc_a1_b1 partition of abc_a1 for values in (1);
create table abc_a1_b2 partition of abc_a1 for values in (2);
create table abc_a1_b3 partition of abc_a1 for values in (3);
create table abc_a3 partition of ab_c for values in(3) partition by list (b);
create table abc_a3_b1 partition of abc_a3 for values in (1);
create table abc_a3_b2 partition of abc_a3 for values in (2);
create table abc_a3_b3 partition of abc_a3 for values in (3);
deallocate abc_q1;

=# explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 1);
QUERY PLAN
------------------------------------------------------------------------------
Append (actual rows=2 loops=1)
-> Seq Scan on abc_a1_b2 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a1_b3 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a2_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a2_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a3_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a3_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
(13 rows)

postgres=# explain (analyze, costs off, summary off, timing off)
execute abc_q1 (1, 2);
ERROR: partition missing from Append subplans

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#70Beena Emerson
memissemerson@gmail.com
In reply to: Beena Emerson (#69)
Re: [HACKERS] Runtime Partition Pruning

Hi,

The mail was accidently sent before I could complete.

On Tue, Jan 9, 2018 at 2:24 PM, Beena Emerson <memissemerson@gmail.com> wrote:

Hello,

The pruning does not work well with char type:

Case 2: Case with optimizer pruning
drop table ab_c;
create table ab_c (a int not null, b int) partition by list(a);
create table abc_a2 (b int, a int not null) partition by list(b);
create table abc_a2_b1 partition of abc_a2 for values in (1);
create table abc_a2_b2 partition of abc_a2 for values in (2);
create table abc_a2_b3 partition of abc_a2 for values in (3);
alter table ab_c attach partition abc_a2 for values in (2);
create table abc_a1 partition of ab_c for values in(1) partition by list (b);
create table abc_a1_b1 partition of abc_a1 for values in (1);
create table abc_a1_b2 partition of abc_a1 for values in (2);
create table abc_a1_b3 partition of abc_a1 for values in (3);
create table abc_a3 partition of ab_c for values in(3) partition by list (b);
create table abc_a3_b1 partition of abc_a3 for values in (1);
create table abc_a3_b2 partition of abc_a3 for values in (2);
create table abc_a3_b3 partition of abc_a3 for values in (3);
deallocate abc_q1;

Prepared statement is missing:
prepare abc_q1 (int, int) as select a,b from ab_c where a BETWEEN $1
and $2 AND b IN (3, 2);

=# explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 1);
QUERY PLAN
------------------------------------------------------------------------------
Append (actual rows=2 loops=1)
-> Seq Scan on abc_a1_b2 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a1_b3 (actual rows=1 loops=1)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a2_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a2_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a3_b2 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
-> Seq Scan on abc_a3_b3 (never executed)
Filter: ((a >= $1) AND (a <= $2) AND (b = ANY ('{3,2}'::integer[])))
(13 rows)

postgres=# explain (analyze, costs off, summary off, timing off)
execute abc_q1 (1, 2);
ERROR: partition missing from Append subplans

These work fine when the column order of subpartitons are not changed.

Case 3: Optimizer pruning with char types:
Same as case1 with all subpartitions having same col order as parent.

drop table ab_c;
create table ab_c (a int not null, b char) partition by list(a);
create table abc_a2 ( a int not null, b char) partition by list(b);
create table abc_a2_b1 partition of abc_a2 for values in ('1');
create table abc_a2_b2 partition of abc_a2 for values in ('2');
create table abc_a2_b3 partition of abc_a2 for values in ('3');
alter table ab_c attach partition abc_a2 for values in (2);
create table abc_a1 partition of ab_c for values in(1) partition by list (b);
create table abc_a1_b1 partition of abc_a1 for values in ('1');
create table abc_a1_b2 partition of abc_a1 for values in ('2');
create table abc_a1_b3 partition of abc_a1 for values in ('3');
create table abc_a3 partition of ab_c for values in(3) partition by list (b);
create table abc_a3_b1 partition of abc_a3 for values in ('1');
create table abc_a3_b2 partition of abc_a3 for values in ('2');
create table abc_a3_b3 partition of abc_a3 for values in ('3');

deallocate abc_q1;
prepare abc_q1 (int, int) as select a,b from ab_c where a BETWEEN $1
and $2 AND b IN ('3', '2');

-- b4 runtime pruning
=# explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 8);
QUERY PLAN
---------------------------------------------------------------------------
Append (actual rows=0 loops=1)
-> Seq Scan on abc_a1_b2 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
-> Seq Scan on abc_a1_b3 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
-> Seq Scan on abc_a2_b2 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
-> Seq Scan on abc_a2_b3 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
-> Seq Scan on abc_a3_b2 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
-> Seq Scan on abc_a3_b3 (actual rows=0 loops=1)
Filter: ((a >= 1) AND (a <= 8) AND (b = ANY ('{3,2}'::bpchar[])))
(13 rows)

-- after 5 runs

=# explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 1);
ERROR: operator 1057 is not a member of opfamily 1976

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#71David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#70)
Re: [HACKERS] Runtime Partition Pruning

On 9 January 2018 at 22:22, Beena Emerson <memissemerson@gmail.com> wrote:

ERROR: operator 1057 is not a member of opfamily 1976

Thanks for finding these. I'm looking into the above, and the other
ones you've mentioned now.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#72David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#70)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On Tue, Jan 9, 2018 at 2:24 PM, Beena Emerson <memissemerson@gmail.com> wrote:

prepare abc_q1 (int, int, char) as select * from ab_c where a BETWEEN
$1 and $2 AND b <= $3;

--after 5 runs: abc_a2_b3 is not pruned.

This seems to be down to an existing bug. I'm not yet sure if it's
caused by faster_partition_prune_v17, or if it exists in master.
Basically RelOptInfo->partition_rels can contain duplicates for
relations. In your example while debugging make_partition_pruneinfo I
see:

list_nth_int(best_path->partitioned_rels,0)
1
list_nth_int(best_path->partitioned_rels,1)
3
list_nth_int(best_path->partitioned_rels,2)
8
list_nth_int(best_path->partitioned_rels,3)
13
list_nth_int(best_path->partitioned_rels,4)
3
list_nth_int(best_path->partitioned_rels,5)
8
list_nth_int(best_path->partitioned_rels,6)
13

There should only be 4 items in this list, not 7.

make_partition_pruneinfo might have been a bit naive to assume this
couldn't happen, so I've coded it to be a bit more resilient to this
happening. It'll still end up creating another sub-PartitionPruneInfo
and slotting into the same place, but it'll no longer attempt to
translate the prunequals twice... which was what was causing the
problem. I'd been a bit sloppy and assigned the output of
adjust_appendrel_attrs() back to the prunequals which is a parameter
to the function instead of assigning to a local variable like I've
done now.

On 9 January 2018 at 22:22, Beena Emerson <memissemerson@gmail.com> wrote:

postgres=# explain (analyze, costs off, summary off, timing off)
execute abc_q1 (1, 2);
ERROR: partition missing from Append subplans

This also seems to be fixed by the above fix.

=# explain (analyze, costs off, summary off, timing off) execute abc_q1 (1, 1);
ERROR: operator 1057 is not a member of opfamily 1976

This seems to be broken in faster_partition_prune_v17 where in
classify_partition_bounding_keys() the code properly checks if the
clause matches the partition key for OpExpr, but fails to do the same
for ScalarArrayOpExpr. I'll report to Amit on the thread for that
patch.

I'll also investigate the duplication in RelOptInfo->partition_rels
and report that in another thread.

Can you confirm that case 1 and 2 are working with the attached?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v8.patchapplication/octet-stream; name=runtime_prune_drowley_v8.patchDownload
diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 8066237..9adbb59 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -34,11 +34,13 @@
 #include "catalog/pg_type.h"
 #include "commands/tablecmds.h"
 #include "executor/executor.h"
+#include "executor/nodeSubplan.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "nodes/parsenodes.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planmain.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
@@ -290,25 +292,33 @@ static uint64 compute_hash_value(PartitionKey key, Datum *values, bool *isnull);
 PG_FUNCTION_INFO_V1(satisfies_hash_partition);
 
 static Bitmapset *get_partitions_from_clauses_recurse(Relation relation,
-								int rt_index, List *clauses);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *clauses);
 static Bitmapset *get_partitions_from_ne_clauses(Relation relation,
-								List *ne_clauses);
+							   ParamListInfo prmlist,
+							   ExprContext *econtext,
+							   List *ne_clauses);
 static Bitmapset *get_partitions_from_or_clause_args(Relation relation,
-								int rt_index, List *or_clause_args);
+								int rt_index, ParamListInfo prmlist,
+								ExprContext *econtext, List *or_clause_args);
 static int classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys, ParamListInfo prmlist,
+								 ExprContext *econtext, bool *constfalse,
 								 List **or_clauses, List **ne_clauses);
 static void remove_redundant_clauses(PartitionKey partkey,
 						 int partattoff, List *all_clauses,
-						 List **result, bool *constfalse);
+						 List **result, ParamListInfo prmlist,
+						 ExprContext *econtext,bool *constfalse);
 static bool partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result);
 static PartOpStrategy partition_op_strategy(PartitionKey key, PartClause *op,
 					bool *incl);
 static bool partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value);
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value);
 static Bitmapset *get_partitions_for_keys(Relation rel,
 						PartScanKeyInfo *keys);
 static Bitmapset *get_partitions_for_keys_hash(Relation rel,
@@ -1698,6 +1708,7 @@ get_partition_qual_relid(Oid relid)
  */
 Bitmapset *
 get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses)
 {
 	Bitmapset	   *result;
@@ -1727,16 +1738,145 @@ get_partitions_from_clauses(Relation relation, int rt_index,
 		if (partition_bound_has_default(boundinfo))
 		{
 			partconstr = (List *) expression_planner((Expr *) partconstr);
-			partclauses = list_concat(partclauses, partconstr);
+			partclauses = list_concat(list_copy(partclauses), partconstr);
 		}
 	}
 
-	result = get_partitions_from_clauses_recurse(relation, rt_index,
-												 partclauses);
+	result = get_partitions_from_clauses_recurse(relation, rt_index, prmlist,
+												 econtext, partclauses);
 
 	return result;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	pinfo->nparts = nparts;
+
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Determine the element in part_appinfo which belongs to this
+		 * subpath.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+
+			if (rel->part_appinfos[partidx]->child_relid !=
+				appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			List *subprunequal;
+
+			if (rel->part_appinfos[partidx]->child_relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			subprunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																	subprunequal);
+			break;
+		}
+	}
+
+	return pinfo;
+}
+
 /* Module-local functions */
 
 /*
@@ -1748,6 +1888,8 @@ get_partitions_from_clauses(Relation relation, int rt_index,
  */
 static Bitmapset *
 get_partitions_from_clauses_recurse(Relation relation, int rt_index,
+									ParamListInfo prmlist,
+									ExprContext *econtext,
 									List *clauses)
 {
 	PartitionDesc partdesc = RelationGetPartitionDesc(relation);
@@ -1764,8 +1906,9 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	 * can work with.
 	 */
 	nkeys = classify_partition_bounding_keys(relation, clauses, rt_index,
-											 &keys, &constfalse,
-											 &or_clauses, &ne_clauses);
+											 &keys, prmlist, econtext,
+											 &constfalse, &or_clauses,
+											 &ne_clauses);
 
 	/*
 	 * classify_partition_bounding_keys() may have found clauses marked
@@ -1799,7 +1942,10 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 	{
 		Bitmapset *ne_clause_parts;
 
-		ne_clause_parts = get_partitions_from_ne_clauses(relation, ne_clauses);
+		ne_clause_parts = get_partitions_from_ne_clauses(relation,
+														 prmlist,
+														 econtext,
+														 ne_clauses);
 
 		/*
 		 * Clauses in ne_clauses are in conjunction with the clauses that gave
@@ -1819,6 +1965,8 @@ get_partitions_from_clauses_recurse(Relation relation, int rt_index,
 		Bitmapset *or_parts;
 
 		or_parts = get_partitions_from_or_clause_args(relation, rt_index,
+													  prmlist,
+													  econtext,
 													  or->args);
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -1889,7 +2037,8 @@ count_partition_datums(Relation rel, int index)
  * ne_clauses.  Only ever called if relation is a list partitioned table.
  */
 static Bitmapset *
-get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
+get_partitions_from_ne_clauses(Relation relation, ParamListInfo prmlist,
+							   ExprContext *econtext, List *ne_clauses)
 {
 	ListCell   *lc;
 	Bitmapset  *result,
@@ -1924,7 +2073,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
 		PartClause *pc = lfirst(lc);
 		Datum	datum;
 
-		if (partkey_datum_from_expr(partkey, 0, pc->constarg, &datum) &&
+		if (partkey_datum_from_expr(partkey, 0, pc->constarg, prmlist,
+			econtext, &datum) &&
 			!datum_in_array(partkey, datum, exclude_datums, n_exclude_datums))
 			exclude_datums[n_exclude_datums++] = datum;
 	}
@@ -1992,6 +2142,8 @@ get_partitions_from_ne_clauses(Relation relation, List *ne_clauses)
  */
 static Bitmapset *
 get_partitions_from_or_clause_args(Relation relation, int rt_index,
+								   ParamListInfo prmlist,
+								   ExprContext *econtext,
 								   List *or_clause_args)
 {
 	ListCell   *lc;
@@ -2024,6 +2176,8 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 		}
 
 		arg_partset = get_partitions_from_clauses_recurse(relation, rt_index,
+														  prmlist,
+														  econtext,
 														  arg_clauses);
 
 		/*
@@ -2079,7 +2233,10 @@ get_partitions_from_or_clause_args(Relation relation, int rt_index,
 static int
 classify_partition_bounding_keys(Relation relation, List *clauses,
 								 int rt_index,
-								 PartScanKeyInfo *keys, bool *constfalse,
+								 PartScanKeyInfo *keys,
+								 ParamListInfo prmlist,
+								 ExprContext *econtext,
+								 bool *constfalse,
 								 List **or_clauses,
 								 List **ne_clauses)
 {
@@ -2384,10 +2541,10 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				elem_clauses = NIL;
 				foreach(lc1, elem_exprs)
 				{
-					Const  *rightop = castNode(Const, lfirst(lc1));
+					Expr   *rightop = (Expr *) lfirst(lc1);
 					Expr   *elem_clause;
 
-					if (rightop->constisnull)
+					if (IsA(rightop, Const) && ((Const *) rightop)->constisnull)
 					{
 						NullTest *nulltest = makeNode(NullTest);
 
@@ -2507,7 +2664,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 	{
 		remove_redundant_clauses(partkey, i,
 								 keyclauses_all[i], &keyclauses[i],
-								 constfalse);
+								 prmlist, econtext, constfalse);
 		if (*constfalse)
 			return 0;
 	}
@@ -2570,11 +2727,13 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 					Assert(incl);
 					if (need_next_eq &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2583,6 +2742,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2593,6 +2753,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_LESS:
 					if (need_next_max &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -2605,6 +2766,7 @@ classify_partition_bounding_keys(Relation relation, List *clauses,
 				case PART_OP_GREATER:
 					if (need_next_min &&
 						partkey_datum_from_expr(partkey, i, constarg,
+												prmlist, econtext,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -2698,7 +2860,8 @@ partition_op_strategy(PartitionKey key, PartClause *op, bool *incl)
  */
 static bool
 partkey_datum_from_expr(PartitionKey key, int partattoff,
-						Expr *expr, Datum *value)
+						Expr *expr, ParamListInfo prmlist,
+						ExprContext *econtext, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -2740,6 +2903,43 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 		case T_Const:
 			*value = ((Const *) expr)->constvalue;
 			return true;
+		case T_Param:
+
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						node = eval_const_expressions_from_list(prmlist,
+															 (Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan(prm->execPlan, econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+
+				default:
+					return false;
+			}
 
 		default:
 			return false;
@@ -2758,6 +2958,7 @@ partkey_datum_from_expr(PartitionKey key, int partattoff,
 static void
 remove_redundant_clauses(PartitionKey partkey, int partattoff,
 						 List *all_clauses, List **result,
+						 ParamListInfo prmlist, ExprContext *econtext,
 						 bool *constfalse)
 {
 	PartClause *hash_clause,
@@ -2801,7 +3002,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			/* check if another clause would contradict the one we have */
 			else if (partition_cmp_args(partkey, partattoff,
 										cur, cur, hash_clause,
-										&test_result))
+										prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2856,7 +3057,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   cur, cur, btree_clauses[s],
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				/* cur is more restrictive, replace old key. */
 				if (test_result)
@@ -2912,7 +3113,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 			 */
 			if (partition_cmp_args(partkey, partattoff,
 								   chk, eq, chk,
-								   &test_result))
+								   prmlist, econtext, &test_result))
 			{
 				if (!test_result)
 				{
@@ -2942,7 +3143,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   le, lt, le,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTLessEqualStrategyNumber - 1] = NULL;
@@ -2960,7 +3161,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 
 		if (partition_cmp_args(partkey, partattoff,
 							   ge, gt, ge,
-							   &test_result))
+							   prmlist, econtext, &test_result))
 		{
 			if (test_result)
 				btree_clauses[BTGreaterEqualStrategyNumber - 1] = NULL;
@@ -2994,6 +3195,7 @@ remove_redundant_clauses(PartitionKey partkey, int partattoff,
 static bool
 partition_cmp_args(PartitionKey key, int partattoff,
 				   PartClause *op, PartClause *leftarg, PartClause *rightarg,
+				   ParamListInfo prmlist, ExprContext *econtext,
 				   bool *result)
 {
 	Oid		partopfamily = key->partopfamily[partattoff];
@@ -3003,10 +3205,12 @@ partition_cmp_args(PartitionKey key, int partattoff,
 	Assert(op->valid_cache && leftarg->valid_cache && rightarg->valid_cache);
 	/* Get the constant values from the operands */
 	if (!partkey_datum_from_expr(key, partattoff,
-								 leftarg->constarg, &leftarg_const))
+								 leftarg->constarg, prmlist, econtext,
+								 &leftarg_const))
 		return false;
 	if (!partkey_datum_from_expr(key, partattoff,
-								 rightarg->constarg, &rightarg_const))
+								 rightarg->constarg, prmlist, econtext,
+								 &rightarg_const))
 		return false;
 
 	/*
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 64a17fb..9a0f49d 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,7 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -82,6 +83,12 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,20 +134,46 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = node->part_prune_params;
+	appendstate->part_prune_info = node->part_prune_info;
 
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
 
 	/*
-	 * append nodes still have Result slots, which hold pointers to tuples, so
-	 * we have to initialize them.
+	 * tuple table initialization
 	 */
 	ExecInitResultTupleSlot(estate, &appendstate->ps);
 
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid now.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
+
 	/*
 	 * call ExecInitNode on each of the plans to be executed and save the
 	 * results into the array "appendplans".
@@ -160,13 +193,8 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecAssignResultTypeFromTL(&appendstate->ps);
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -257,6 +285,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -276,8 +316,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -366,22 +406,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -414,6 +467,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -466,6 +530,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +600,111 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   Bitmapset **validsubplans)
+{
+	Bitmapset	   *partset;
+	Relation		rel;
+	int				i;
+
+	rel = relation_open(pinfo->parentoid, NoLock);
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(rel, 1,
+										  node->ps.state->es_param_list_info,
+										  node->ps.ps_ExprContext,
+										  pinfo->prunequal);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+
+	relation_close(rel, NoLock);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index 733fe3c..6afc90e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1045,6 +1067,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ddbbc79..be60bba 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -243,6 +243,8 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_BITMAPSET_FIELD(part_prune_params);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2126,6 +2128,32 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(PartitionPruneInfo));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			newnode->subpartindex[i] =
+							_copyPartitionPruneInfo(newnode->subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5007,6 +5035,9 @@ copyObjectImpl(const void *from)
 		case T_OnConflictExpr:
 			retval = _copyOnConflictExpr(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * RELATION NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 6774e08..e52dcb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -141,12 +141,6 @@ static void add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 static List *get_append_rel_partitions(PlannerInfo *root,
 						  RelOptInfo *rel,
 						  RangeTblEntry *rte);
-static List *match_clauses_to_partkey(PlannerInfo *root,
-						 RelOptInfo *rel,
-						 List *clauses,
-						 bool *contains_const,
-						 bool *constfalse);
-
 
 /*
  * make_one_rel
@@ -877,6 +871,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	Relation		parent;
 	PartitionDesc	partdesc;
 	Bitmapset	   *partindexes;
+	Bitmapset	   *paramids = NULL;
 
 	/*
 	 * Get the clauses that match the partition key, including information
@@ -885,6 +880,7 @@ get_append_rel_partitions(PlannerInfo *root,
 	 */
 	partclauses = match_clauses_to_partkey(root, rel,
 										   list_copy(rel->baserestrictinfo),
+										   &paramids,
 										   &contains_const,
 										   &constfalse);
 
@@ -892,6 +888,13 @@ get_append_rel_partitions(PlannerInfo *root,
 	if (constfalse)
 		return NIL;
 
+	/*
+	 * Record any params found that we could use to further eliminate
+	 * partitions during execution.
+	 */
+	rel->runtime_prune_params = bms_add_members(rel->runtime_prune_params,
+												paramids);
+
 	parent = heap_open(rte->relid, NoLock);
 	partdesc = RelationGetPartitionDesc(parent);
 
@@ -900,8 +903,8 @@ get_append_rel_partitions(PlannerInfo *root,
 	 * then use these to prune partitions.
 	 */
 	if (partclauses != NIL && contains_const)
-		partindexes = get_partitions_from_clauses(parent, rel->relid,
-												  partclauses);
+		partindexes = get_partitions_from_clauses(parent, rel->relid, NULL,
+												  NULL, partclauses);
 	else
 	{
 		/*
@@ -963,10 +966,11 @@ get_append_rel_partitions(PlannerInfo *root,
  * If the list contains a pseudo-constant RestrictInfo with constant false
  * value, *constfalse is set.
  */
-static List *
+List *
 match_clauses_to_partkey(PlannerInfo *root,
 						 RelOptInfo *rel,
 						 List *clauses,
+						 Bitmapset **paramids,
 						 bool *contains_const,
 						 bool *constfalse)
 {
@@ -1030,6 +1034,7 @@ match_clauses_to_partkey(PlannerInfo *root,
 							constfalse1;
 
 					if (match_clauses_to_partkey(root, rel, list_make1(arg),
+												 paramids,
 												 &contains_const1,
 												 &constfalse1) != NIL)
 					{
@@ -1154,8 +1159,11 @@ match_clauses_to_partkey(PlannerInfo *root,
 				 */
 				result = lappend(result, clause);
 
-				if (!*contains_const)
-					*contains_const = IsA(constexpr, Const);
+				if (IsA(constexpr, Const))
+					*contains_const = true;
+				else if (IsA(constexpr, Param))
+					*paramids = bms_add_member(*paramids,
+										   ((Param *) constexpr)->paramid);
 			}
 			else if (IsA(clause, ScalarArrayOpExpr))
 			{
@@ -1187,10 +1195,32 @@ match_clauses_to_partkey(PlannerInfo *root,
 
 				/* OK to add to the result. */
 				result = lappend(result, clause);
-				if (IsA(estimate_expression_value(root, rightop), Const))
+
+				if (IsA(rightop, Const))
 					*contains_const = true;
-				else
-					*contains_const = false;
+				else if (IsA(rightop, ArrayExpr))
+				{
+					ArrayExpr *arrayexpr = (ArrayExpr *) rightop;
+					ListCell   *lc;
+					bool		allconsts = true;
+
+					foreach(lc, arrayexpr->elements)
+					{
+						Expr *expr = (Expr *) lfirst(lc);
+
+						if (IsA(expr, Const))
+							continue;
+
+						allconsts = false;
+
+						if (IsA(expr, Param))
+							*paramids = bms_add_member(*paramids,
+												   ((Param *) expr)->paramid);
+					}
+
+					if (allconsts)
+						*contains_const = true;
+				}
 			}
 			else if (IsA(clause, NullTest))
 			{
@@ -1907,7 +1937,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1949,8 +1979,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1998,7 +2028,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -2054,7 +2084,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2319,7 +2349,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index d9249f4..1bd92c1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1232,7 +1232,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e599283..5b52480 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -204,7 +204,9 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1016,6 +1018,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1053,6 +1057,53 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+			bool		contains_const;
+			bool		constfalse;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+
+			/*
+			 * So far, we only know about the pruning params for the base quals
+			 * in rel, there may well be params matching partition keys in the
+			 * parameterized path clause too, so we'll gather these now. We'll
+			 * borrow match_clauses_to_partkey for this, although we only care
+			 * about the parameter IDs and not any of the other outputs.
+			 */
+			(void) match_clauses_to_partkey(root, rel, prmquals,
+											&rel->runtime_prune_params,
+											&contains_const, &constfalse);
+		}
+
+		/*
+		 * If there are parameters matching the partition key then we'll now
+		 * enable run-time partition pruning.  There's no fancy big switch
+		 * to enable it, we'll just make a PartitionPruneInfo and pass that
+		 * along to the executor. It'll just make use of it when available.
+		 */
+		if (rel->runtime_prune_params)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1061,7 +1112,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo,
+					   rel->runtime_prune_params);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5308,7 +5361,9 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo,
+			Bitmapset *partpruneparams)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5320,7 +5375,8 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
+	node->part_prune_params = partpruneparams;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b0f6051..0eff96a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3678,7 +3678,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 5a08e75..64a981e 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -590,7 +590,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -702,7 +702,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ccfae4f..8322665 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 7df8761..43d1fa3 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,37 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths, doing anything extra in this case would be wasted
+	 * work.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c
index 4b5d50e..e98f4d7 100644
--- a/src/backend/optimizer/util/relnode.c
+++ b/src/backend/optimizer/util/relnode.c
@@ -156,6 +156,7 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
 	rel->boundinfo = NULL;
 	rel->part_appinfos = NULL;
 	rel->part_rels = NULL;
+	rel->runtime_prune_params = NULL;
 	rel->partexprs = NULL;
 	rel->nullable_partexprs = NULL;
 	rel->live_part_appinfos = NIL;
@@ -577,6 +578,7 @@ build_join_rel(PlannerInfo *root,
 	joinrel->boundinfo = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
@@ -745,6 +747,7 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
 	joinrel->part_scheme = NULL;
 	joinrel->part_appinfos = NULL;
 	joinrel->part_rels = NULL;
+	joinrel->runtime_prune_params = NULL;
 	joinrel->partexprs = NULL;
 	joinrel->nullable_partexprs = NULL;
 
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8423c6e..cb49193 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -75,5 +76,10 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 
 /* For partition-pruning */
 extern Bitmapset *get_partitions_from_clauses(Relation relation, int rt_index,
+							ParamListInfo prmlist, ExprContext *econtext,
 							List *partclauses);
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 15397e9..951ddb8 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -98,6 +98,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 2a4f740..f984769 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1000,6 +1000,7 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
@@ -1016,6 +1017,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 2eb3d6d..9788c35 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 74e9fb5..1d57748 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -249,6 +249,14 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+	Bitmapset  *part_prune_params; /* ParamIds used for partition pruning */
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..915f675 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,25 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			parentoid; /* Oid of parent partition rel */
+	List	   *prunequal; /* qual list for pruning partitions */
+	int			nparts; /* length of the following arrays */
+	int		   *subnodeindex; /* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 855d51e..0f0bc52 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -669,6 +669,9 @@ typedef struct RelOptInfo
 	struct RelOptInfo **part_rels;	/* Array of RelOptInfos of *all*
 									 * partitions, stored in the same order as
 									 * of bounds */
+	Bitmapset   *runtime_prune_params;	/* Only valid for base partition rels.
+										 * Stores ParamIds used for run-time
+										 * pruning of partitions. */
 	List	  **partexprs;		/* Non-nullable partition key expressions. */
 	List	  **nullable_partexprs; /* Nullable partition key expressions. */
 
@@ -1295,6 +1298,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9b4288a..44f34a9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 0072b7a..8175c3f 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -50,6 +50,12 @@ extern PGDLLIMPORT join_search_hook_type join_search_hook;
 
 extern RelOptInfo *make_one_rel(PlannerInfo *root, List *joinlist);
 extern void set_dummy_rel_pathlist(RelOptInfo *rel);
+extern List *match_clauses_to_partkey(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *clauses,
+						 Bitmapset **paramids,
+						 bool *contains_const,
+						 bool *constfalse);
 extern RelOptInfo *standard_join_search(PlannerInfo *root, int levels_needed,
 					 List *initial_rels);
 
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ad29f0f..edb3168 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1427,3 +1427,959 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6921e39..81ee321 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -225,3 +225,255 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partitioning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
#73David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#72)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hi,

I've attached an updated patch, now at v10. v9 was short lived due to
the evolution of Amit's which which this based on.

This version is based on Amit's v27 of faster partition pruning [1]/messages/by-id/520f8a71-286d-e36d-34cf-363fd74366e1@lab.ntt.co.jp
which can be applied atop of ad7dbee36.

I've done a self review of this, but I've not yet done any final
polishing work as the faster partition pruning patch is still
evolving. I will, for example, likely need to do some more work in
nodeAppend.c to add a few more comments and probably think of better
names for a few new things that have made a first appearance in this
version of the patch

[1]: /messages/by-id/520f8a71-286d-e36d-34cf-363fd74366e1@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v10.patchapplication/octet-stream; name=runtime_prune_drowley_v10.patchDownload
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7a3dd2e..401ed5d 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,9 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "optimizer/partprune.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -76,12 +79,25 @@ struct ParallelAppendState
 	bool		pa_finished[FLEXIBLE_ARRAY_MEMBER];
 };
 
+struct PartitionPruneContextCache
+{
+	PartitionPruneContext *context;
+	PartitionPruneContextCache *subcache;
+};
+
 #define INVALID_SUBPLAN_INDEX		-1
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +143,34 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = NULL; /* determined later */
+	appendstate->part_prune_info = node->part_prune_info;
+	appendstate->contextcache = NULL; /* populate this as needed below */
+
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
@@ -149,18 +193,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +291,18 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (node->part_prune_info &&
+		bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +322,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,22 +412,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -408,6 +473,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -460,6 +536,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +612,210 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	if (!node->contextcache)
+		node->contextcache = palloc0(sizeof(PartitionPruneContextCache));
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   node->contextcache,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context;
+	Bitmapset	   *partset;
+	int				i;
+
+	check_stack_depth();
+
+	/*
+	 * If the PartitionPruneContext has not yet been initialized for this rel
+	 * yet, then do that now.
+	 */
+	if (!ctxcache->context)
+	{
+		PartitionDesc partdesc;
+		Relation		rel;
+		PartitionKey	partkey;
+		ListCell	   *lc;
+		int				i;
+		MemoryContext oldContext;
+
+		oldContext = MemoryContextSwitchTo(node->ps.state->es_query_cxt);
+
+		ctxcache->context = context = palloc(sizeof(PartitionPruneContext));
+		ctxcache->subcache = palloc0(sizeof(PartitionPruneContextCache) *
+									 pinfo->nparts);
+
+		rel = relation_open(pinfo->parentoid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->relid = pinfo->relid;
+		context->strategy = partkey->strategy;
+		context->partnatts = partkey->partnatts;
+		context->partkeys = palloc(sizeof(Expr *) * context->partnatts);
+
+		lc = list_head(partkey->partexprs);
+
+		for (i = 0; i < context->partnatts; i++)
+		{
+			AttrNumber	attno = partkey->partattrs[i];
+
+			if (attno != InvalidAttrNumber)
+			{
+				Assert(attno > 0);
+
+				context->partkeys[i] = (Expr *) makeVar(pinfo->relid,
+														attno,
+														partkey->parttypid[i],
+													partkey->parttypmod[i],
+													partkey->parttypcoll[i],
+														0);
+			}
+			else
+			{
+				if (lc == NULL)
+					elog(ERROR, "wrong number of partition key expressions");
+
+				context->partkeys[i] = (Expr *) lfirst(lc);
+				lc = lnext(lc);
+			}
+		}
+
+		context->parttypid = partkey->parttypid;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		if (OidIsValid(get_default_oid_from_partdesc(partdesc)))
+			context->has_default_part = true;
+		else
+			context->has_default_part = false;
+
+		context->partition_qual = RelationGetPartitionQual(rel);
+
+		context->prmlist = node->ps.state->es_param_list_info;
+		context->econtext = node->ps.ps_ExprContext;
+		context->paramids = NULL;
+
+		generate_partition_clauses(context, pinfo->prunequal);
+
+		node->part_prune_params = bms_add_members(node->part_prune_params,
+												  context->paramids);
+
+		relation_close(rel, NoLock);
+
+		MemoryContextSwitchTo(oldContext);
+	}
+	else
+		context = ctxcache->context;
+
+	/*
+	 * Detect if any impossibilities were discovered during
+	 * generate_partition_clauses
+	 */
+	if (context->clauseinfo->constfalse)
+	{
+		bms_free(*validsubplans);
+		*validsubplans = NULL;
+		return;
+	}
+
+	/* Determine which partition indexes we need to scan */
+	partset = get_partitions_from_clauses(context);
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node, pinfo->subpartindex[i],
+											   &ctxcache->subcache[i],
+											   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+	bms_free(partset);
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a..3578c8f 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d844b8b..1dc7651 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -244,6 +244,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2151,6 +2152,33 @@ _copyPartitionClauseInfo(const PartitionClauseInfo *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(relid);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts *
+					   sizeof(PartitionPruneInfo *));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			COPY_NODE_FIELD(subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5049,6 +5077,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index d40429a..8a02887 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1587,7 +1587,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1629,8 +1629,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1678,7 +1678,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1734,7 +1734,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2000,7 +2000,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3..2e289d4 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index da0cc7f..d82fee7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -204,7 +205,8 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1022,6 +1024,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1059,6 +1063,41 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist that could possibly be useful to use for
+		 * performing further partition pruning during execution, then
+		 * we'll generate a PartitionPruneInfo to store these quals and
+		 * allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1067,7 +1106,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5320,7 +5360,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5332,7 +5373,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 646d118..f720e8d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3717,7 +3717,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f01119e..146e202 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -593,7 +593,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -705,7 +705,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 0c1f239..dccbcc1 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 71a7b7b..683dd5d 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -56,12 +56,16 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/nodeSubplan.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
+#include "optimizer/prep.h"
 #include "parser/parse_coerce.h"
 #include "parser/parsetree.h"
 #include "rewrite/rewriteManip.h"
@@ -101,14 +105,15 @@ static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
 								   List *or_args);
 static void remove_redundant_clauses(PartitionPruneContext *context,
 						 List **minimalclauses);
-static bool partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result);
+static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
-static bool partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value);
+static bool partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value);
 
 /*
  * prune_append_rel_partitions
@@ -154,6 +159,9 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.boundinfo = rel->boundinfo;
 		context.has_default_part = rel->has_default_part;
 		context.partition_qual = rel->partition_qual;
+		context.prmlist = NULL;
+		context.econtext = NULL;
+		context.paramids = NULL;
 
 		/* process clauses; context.clauseinfo will be set */
 		generate_partition_clauses(&context, clauses);
@@ -487,6 +495,10 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				pc->inputcollid = opclause->inputcollid;
 				pc->value = valueexpr;
 
+				if (IsA(valueexpr, Param))
+					context->paramids = bms_add_member(context->paramids,
+											((Param *) valueexpr)->paramid);
+
 				/*
 				 * We don't turn a <> operator clause into a key right away.
 				 * Instead, the caller will hand over such clauses to
@@ -641,6 +653,11 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 														 leftop, rightop,
 														 InvalidOid,
 														 saop_coll);
+
+					if (IsA(rightop, Param))
+						context->paramids = bms_add_member(context->paramids,
+												((Param *) rightop)->paramid);
+
 					elem_clauses = lappend(elem_clauses, elem_clause);
 				}
 
@@ -891,7 +908,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				if (hash_clause == NULL)
 					hash_clause = pc;
 				/* check if another clause would contradict the one we have */
-				else if (partition_cmp_args(context->parttypid[i],
+				else if (partition_cmp_args(context,
+											context->parttypid[i],
 											context->partopfamily[i],
 											pc, pc, hash_clause,
 											&test_result))
@@ -948,7 +966,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * then because 7 < 5 is false, we leave a < 5 where it is and
 				 * effectively discard a < 7 as being redundant.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   pc, pc, btree_clauses[s],
 									   &test_result))
@@ -1005,7 +1024,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * eq clause is a = 3, then because 3 < 5, we no longer need
 				 * a < 5, because a = 3 is more restrictive.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   chk, eq, chk,
 									   &test_result))
@@ -1036,7 +1056,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *lt = btree_clauses[BTLessStrategyNumber - 1],
 					   *le = btree_clauses[BTLessEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   le, lt, le,
 								   &test_result))
@@ -1055,7 +1076,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *gt = btree_clauses[BTGreaterStrategyNumber - 1],
 					   *ge = btree_clauses[BTGreaterEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   ge, gt, ge,
 								   &test_result))
@@ -1093,9 +1115,9 @@ remove_redundant_clauses(PartitionPruneContext *context,
  * incompatible with the operator.
  */
 static bool
-partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result)
+partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result)
 {
 	Datum	left_value;
 	Datum	right_value;
@@ -1106,10 +1128,12 @@ partition_cmp_args(Oid parttypid, Oid partopfamily,
 	 * Try to extract an actual value from each arg.  This may fail if the
 	 * value is unknown in this context, in which case we cannot compare.
 	 */
-	if (!partkey_datum_from_expr(parttypid, leftarg->value, &left_value))
+	if (!partkey_datum_from_expr(context, parttypid, leftarg->value,
+		&left_value))
 		return false;
 
-	if (!partkey_datum_from_expr(parttypid, rightarg->value, &right_value))
+	if (!partkey_datum_from_expr(context, parttypid, rightarg->value,
+		&right_value))
 		return false;
 
 	/*
@@ -1232,12 +1256,16 @@ extract_bounding_datums(PartitionPruneContext *context,
 				case PART_OP_EQUAL:
 					Assert(incl);
 					if (need_next_eq &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1245,7 +1273,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 					}
 
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1255,7 +1285,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_LESS:
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1267,7 +1299,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_GREATER:
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1314,8 +1348,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 			PartClause *pc = (PartClause *) lfirst(lc);
 			Datum	datum;
 
-			if (partkey_datum_from_expr(context->parttypid[0], pc->value,
-										&datum))
+			if (partkey_datum_from_expr(context, context->parttypid[0],
+										pc->value, &datum))
 				keys->ne_datums[i++] = datum;
 		}
 		keys->n_ne_datums = i;
@@ -1391,7 +1425,8 @@ partition_op_strategy(char part_strategy, PartClause *pc, bool *incl)
  * set.  True is returned otherwise.
  */
 static bool
-partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -1429,11 +1464,177 @@ partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
 	 * Add more expression types here as needed to support the requirements
 	 * of the higher-level code.
 	 */
-	if (IsA(expr, Const))
+	switch (nodeTag(expr))
 	{
-		*value = ((Const *) expr)->constvalue;
-		return true;
+		case T_Const:
+			*value = ((Const *) expr)->constvalue;
+			return true;
+
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (context->prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						ParamListInfo prmlist = context->prmlist;
+
+						node = eval_const_expressions_from_list(prmlist,
+																(Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (context->econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+						ExprContext *econtext = context->econtext;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan((SubPlanState *) prm->execPlan,
+											 econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+			}
 	}
 
 	return false;
 }
+
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes.  This is required in order to allow
+ *		us to perform any further partition pruning during execution.
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	check_stack_depth();
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->relid = rel->relid;
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->nparts = nparts;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/* Determine the element in part_rel which belongs to this subpath. */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			List *subprunequal;
+
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			subprunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																	subprunequal);
+			break;
+		}
+	}
+
+	return pinfo;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fe3b458..f3ab0d9 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,36 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time. Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths. We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3596,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 0dd6bd3..27d70fc 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -70,6 +71,11 @@ typedef struct PartitionPruneContext
 
 	/* Information about matched clauses */
 	PartitionClauseInfo *clauseinfo;
+	ParamListInfo prmlist;
+	ExprContext *econtext;
+
+	/* ParamIds of clauses being used to determine partitions */
+	Bitmapset *paramids;
 } PartitionPruneContext;
 
 /*
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920..b6f1a9e 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index a953820..7db3a79 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1010,11 +1010,14 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
 struct AppendState;
 typedef struct AppendState AppendState;
+struct PartitionPruneContextCache;
+typedef struct PartitionPruneContextCache PartitionPruneContextCache;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
 
@@ -1026,6 +1029,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	PartitionPruneContextCache *contextcache; /* cache of prune contexts */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 81d223c..1ca7d64 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -192,6 +192,7 @@ typedef enum NodeTag
 	T_OnConflictExpr,
 	T_PartitionClauseInfo,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f2e19ea..a3aeb95 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -250,6 +250,13 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition numbers into subplan indexes. This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 6cfb876..9f538d2 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1546,4 +1546,26 @@ typedef struct PartitionClauseInfo
 	bool	constfalse;
 } PartitionClauseInfo;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * id number into an Append node's subplan index.  This structure is used
+ * to recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	int			relid;		/* relation index of parent partition rel */
+	Oid			parentoid;	/* Oid of parent partition rel */
+	List	   *prunequal;	/* qual list for pruning partitions */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 5579940..4d17f9a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1294,6 +1294,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 5c0d469..57701e4 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -22,4 +22,9 @@ extern void generate_partition_clauses(PartitionPruneContext *context,
 							List *clauses);
 extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context);
 
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index ef7173f..bde1858 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index bc9ff38..478820f 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1419,3 +1419,959 @@ explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' a
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index b7c5abf..f66c193 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -228,3 +228,255 @@ explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
 explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp;
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
#74Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: David Rowley (#73)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Feb 17, 2018 at 2:27 PM, David Rowley <david.rowley@2ndquadrant.com>
wrote:

Hi,

I've attached an updated patch, now at v10. v9 was short lived due to
the evolution of Amit's which which this based on.

This version is based on Amit's v27 of faster partition pruning [1]
which can be applied atop of ad7dbee36.

Hi,

I have applied v10 patch on Amit's v27 over head ad7dbee36. I got "ERROR:
partition missing from Append subplans" with the patch. on head and only
with Amit's patches query is working fine, Please find test case below.

CREATE TABLE part ( c1 INT2, c2 DATE) PARTITION BY RANGE (c1);
CREATE TABLE part_p1 PARTITION OF part FOR VALUES FROM (0) TO (141)
PARTITION BY RANGE(c2);
CREATE TABLE part_p11 PARTITION OF part_p1 FOR VALUES FROM ('1/1/1997') TO
('2/1/1999');
CREATE TABLE part_p12 PARTITION OF part_p1 FOR VALUES FROM ('2/1/1999') TO
('2/1/2000');
CREATE TABLE part_p2 PARTITION OF part FOR VALUES FROM (141) TO (211)
PARTITION BY RANGE(c2);
CREATE TABLE part_p21 PARTITION OF part_p2 FOR VALUES FROM ('1/1/2000') TO
('2/1/2001');
CREATE TABLE part_p22 PARTITION OF part_p2 FOR VALUES FROM ('2/1/2001') TO
('2/1/2006');

INSERT INTO part VALUES (100,'1/1/1999');
INSERT INTO part VALUES (110,'1/1/1998');
INSERT INTO part VALUES (130,'1/1/2000');
INSERT INTO part VALUES (170,'1/1/2000');
INSERT INTO part VALUES (180,'1/1/2001');
INSERT INTO part VALUES (190,'1/1/2006');
INSERT INTO part VALUES (200,'1/1/2000');
INSERT INTO part VALUES (210,'1/1/2002');

postgres=# PREPARE RTP AS SELECT * FROM PART WHERE c2 BETWEEN '1/1/1998'
AND '1/1/1999';
PREPARE
postgres=# EXPLAIN execute RTP;
QUERY
PLAN
-----------------------------------------------------------------------------
Append (cost=0.00..46.00 rows=12 width=6)
-> Seq Scan on part_p11 (cost=0.00..46.00 rows=12 width=6)
Filter: ((c2 >= '1998-01-01'::date) AND (c2 <= '1999-01-01'::date))
(3 rows)

postgres=# execute RTP;
*ERROR: partition missing from Append subplans*

deallocate RTP;
DROP TABLE part;

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

#75David Rowley
david.rowley@2ndquadrant.com
In reply to: Rajkumar Raghuwanshi (#74)
Re: [HACKERS] Runtime Partition Pruning

On 20 February 2018 at 23:46, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

I have applied v10 patch on Amit's v27 over head ad7dbee36. I got "ERROR:
partition missing from Append subplans" with the patch. on head and only
with Amit's patches query is working fine, Please find test case below.

Thanks for the test case. I can recreate locally. This is down to the
fact that make_partition_pruneinfo() only makes sub-PartitionPruneInfo
for actual subpaths found in the Append. Your test case happens to
match both the part_p1 and part_p2 partitions on the first level of
iteration, but since no child of part_p2 is found in
make_partition_pruneinfo, that element in the subpartindex never gets
set.

The fix might be to just remove the error and silently ignore those
cases, but I was hoping to keep that around as it might catch other
bugs. I'm just not sure yet how to do both.

I'll rebase this on Amit's latest patch and have a think about it
while doing that.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#76David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#75)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 21 February 2018 at 13:44, David Rowley <david.rowley@2ndquadrant.com> wrote:

On 20 February 2018 at 23:46, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

I have applied v10 patch on Amit's v27 over head ad7dbee36. I got "ERROR:
partition missing from Append subplans" with the patch. on head and only
with Amit's patches query is working fine, Please find test case below.

Thanks for the test case. I can recreate locally. This is down to the
fact that make_partition_pruneinfo() only makes sub-PartitionPruneInfo
for actual subpaths found in the Append. Your test case happens to
match both the part_p1 and part_p2 partitions on the first level of
iteration, but since no child of part_p2 is found in
make_partition_pruneinfo, that element in the subpartindex never gets
set.

The fix might be to just remove the error and silently ignore those
cases, but I was hoping to keep that around as it might catch other
bugs. I'm just not sure yet how to do both.

I'll rebase this on Amit's latest patch and have a think about it
while doing that.

I ended up fixing this another way. The patch currently will build a
PartitionPruneInfo and include that in the Append node for any Append
node which belongs to a partitioned table which has a non-empty set of
clauses to attempt to use for pruning. Currently, at no point in
planning does the patch verify those quals to see if they'd be any use
for partition pruning during execution. Really to be any use they
would need to contain at least one Param which is in a clause which
matches a partitioned key of that partition or some subpartition
thereof.

At the moment these clauses are only verified during the first call of
set_valid_runtime_subplans() which is called the first time the
choose_next_subplan() function is called in nodeAppend.c. It would
be possible to determine this during planning, but it would mean doing
an extra call of generate_partition_clauses(), of which most of the
work would be thrown away, as we'd only really want to know if any
Params match the partition key. I thought it was best to do this
during execution as then we can actually make full use of the work
done in that function and cache the result for reuse each time we need
to redetermine the newly matching subplans when any param matching a
partition key changes. I don't entirely think this is perfect, but
since we can't send the PartitionClauseInfo over to the executor from
the planner it seems like the best option.

The way I fixed the reported error was to cache all the subplans for
the partition at the current hierarchy level and if the quals don't
contain any Params matching the partition key, then we can safely
assume that all of the subplans for that partition must match. The
planner should have pruned any unneeded partitions for this
partitioned table since there are no Params, which would only be known
at run-time. Doing it this way allowed me to keep the sanity check
that alerted you to find this bug in the first place.

Other things I don't particularly like about the current patch are how
I had to construct the partition key expressions in
set_valid_runtime_subplans_recurse(). This pretty much uses code
borrowed from set_baserel_partition_key_exprs(). One way around that
would be to change the partprune.c code to deal with the
partkey->partattrs and consume an expression from the list on attnum =
0. I didn't do that as I want to minimise how much I touch Amit's
patch before it gets committed as doing so would likely just cause
more headaches for me keeping this merged with his work. Another
option to resolve this would be to put the partition key expressions
into the PartitionPruneInfo.

I've attached v11 of the patch.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

runtime_prune_drowley_v11.patchapplication/octet-stream; name=runtime_prune_drowley_v11.patchDownload
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7a3dd2e..d4b7c08 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,9 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "optimizer/partprune.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -76,12 +79,25 @@ struct ParallelAppendState
 	bool		pa_finished[FLEXIBLE_ARRAY_MEMBER];
 };
 
+struct PartitionPruneContextCache
+{
+	PartitionPruneContext *context;
+	PartitionPruneContextCache *subcache;
+};
+
 #define INVALID_SUBPLAN_INDEX		-1
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +143,34 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = NULL; /* determined later */
+	appendstate->part_prune_info = node->part_prune_info;
+	appendstate->contextcache = NULL; /* populate this as needed below */
+
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
@@ -149,18 +193,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +291,17 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +321,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,22 +411,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -408,6 +472,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -460,6 +535,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +611,220 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	if (!node->contextcache)
+		node->contextcache = palloc0(sizeof(PartitionPruneContextCache));
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   node->contextcache,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context;
+	Bitmapset	   *partset;
+	int				i;
+
+	check_stack_depth();
+
+	/*
+	 * If the PartitionPruneContext has not yet been initialized for this rel
+	 * yet, then do that now.
+	 */
+	if (!ctxcache->context)
+	{
+		PartitionDesc partdesc;
+		Relation		rel;
+		PartitionKey	partkey;
+		ListCell	   *lc;
+		int				i;
+		MemoryContext oldContext;
+
+		oldContext = MemoryContextSwitchTo(node->ps.state->es_query_cxt);
+
+		ctxcache->context = context = palloc(sizeof(PartitionPruneContext));
+		ctxcache->subcache = palloc0(sizeof(PartitionPruneContextCache) *
+									 pinfo->nparts);
+
+		rel = relation_open(pinfo->parentoid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->relid = pinfo->relid;
+		context->strategy = partkey->strategy;
+		context->partnatts = partkey->partnatts;
+		context->partkeys = palloc(sizeof(Expr *) * context->partnatts);
+
+		lc = list_head(partkey->partexprs);
+
+		for (i = 0; i < context->partnatts; i++)
+		{
+			AttrNumber	attno = partkey->partattrs[i];
+
+			if (attno != InvalidAttrNumber)
+			{
+				Assert(attno > 0);
+
+				context->partkeys[i] = (Expr *) makeVar(pinfo->relid,
+														attno,
+														partkey->parttypid[i],
+													partkey->parttypmod[i],
+													partkey->parttypcoll[i],
+														0);
+			}
+			else
+			{
+				if (lc == NULL)
+					elog(ERROR, "wrong number of partition key expressions");
+
+				context->partkeys[i] = (Expr *) lfirst(lc);
+				lc = lnext(lc);
+			}
+		}
+
+		context->parttypid = partkey->parttypid;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		if (OidIsValid(get_default_oid_from_partdesc(partdesc)))
+			context->has_default_part = true;
+		else
+			context->has_default_part = false;
+
+		context->partition_qual = RelationGetPartitionQual(rel);
+
+		context->prmlist = node->ps.state->es_param_list_info;
+		context->econtext = node->ps.ps_ExprContext;
+		context->paramids = NULL;
+
+		generate_partition_clauses(context, pinfo->prunequal);
+
+		node->part_prune_params = bms_add_members(node->part_prune_params,
+												  context->paramids);
+
+		relation_close(rel, NoLock);
+
+		MemoryContextSwitchTo(oldContext);
+	}
+	else
+		context = ctxcache->context;
+
+	/*
+	 * Detect if any impossibilities were discovered during
+	 * generate_partition_clauses
+	 */
+	if (context->clauseinfo->constfalse)
+	{
+		bms_free(*validsubplans);
+		*validsubplans = NULL;
+		return;
+	}
+
+	/*
+	 * 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 subpartitions as we may find their partitions keys match some
+	 * params.
+	 */
+	if (!bms_is_empty(context->paramids))
+		partset = get_partitions_from_clauses(context);
+	else
+		partset = pinfo->allsubnodes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node,
+												pinfo->subpartindex[i],
+												&ctxcache->subcache[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than what was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a..3578c8f 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 1bb76dd..4e28144 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -244,6 +244,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2132,6 +2133,34 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(relid);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_BITMAPSET_FIELD(allsubnodes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts *
+					   sizeof(PartitionPruneInfo *));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			COPY_NODE_FIELD(subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5027,6 +5056,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 17eae10..fe3c1a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1587,7 +1587,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1629,8 +1629,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1678,7 +1678,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1734,7 +1734,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2000,7 +2000,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3..2e289d4 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index da0cc7f..d82fee7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -204,7 +205,8 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1022,6 +1024,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1059,6 +1063,41 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist that could possibly be useful to use for
+		 * performing further partition pruning during execution, then
+		 * we'll generate a PartitionPruneInfo to store these quals and
+		 * allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1067,7 +1106,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5320,7 +5360,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5332,7 +5373,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 8fa90b1..2e96491 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3719,7 +3719,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f01119e..146e202 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -593,7 +593,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -705,7 +705,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 0c1f239..dccbcc1 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index f945402..133f0c9 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -40,6 +40,11 @@
  * from the clauses and comparing it with the partition bounds while also
  * taking into account strategies of the operators in the matched clauses.
  *
+ * make_partition_pruneinfo()
+ *
+ * Generates a PartitionPruneInfo node for use in the executor to allow it
+ * to perform partition pruning during execution.
+ *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
@@ -55,10 +60,13 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/nodeSubplan.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -68,43 +76,6 @@
 #include "utils/lsyscache.h"
 
 /*
- * Stores clauses which were matched to a partition key.
- *
- * Each matching "operator" clause is stored in the 'keyclauses' list for the
- * partition key that it was matched to, except if the operator is <>, in
- * which case, the clause is added to the 'ne_clauses' list.
- *
- * Boolean OR clauses whose at least one argument clause matches a partition
- * key are added to the 'or_clauses' list.
- *
- * Based on a IS NULL or IS NOT NULL clause that was matched to a partition
- * key, the corresponding bit in 'keyisnull' or 'keyisnotnull' is set.  A bit
- * in 'keyisnotnull' may also be set when a strict OpExpr is encountered for
- * the given partition key.
- */
-typedef struct PartitionClauseInfo
-{
-	/* Lists of clauses indexed by the partition key */
-	List   *keyclauses[PARTITION_MAX_KEYS];
-
-	/* Each members is a List itself of a given OR clauses's arguments. */
-	List   *or_clauses;
-
-	/* List of clauses containing <> operator. */
-	List   *ne_clauses;
-
-	/* Nth (0 <= N < partnatts) bit set if the key is NULL or NOT NULL. */
-	Bitmapset   *keyisnull;
-	Bitmapset   *keyisnotnull;
-
-	/* True if at least one of above fields contains valid information. */
-	bool	foundkeyclauses;
-
-	/* True if mutually contradictory clauses were found. */
-	bool	constfalse;
-} PartitionClauseInfo;
-
-/*
  * Information about a clause matched with a partition key column kept to
  * avoid recomputing it in remove_redundant_clauses().
  */
@@ -140,14 +111,15 @@ static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
 								   List *or_args);
 static void remove_redundant_clauses(PartitionPruneContext *context,
 						 List **minimalclauses);
-static bool partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result);
+static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
-static bool partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value);
+static bool partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value);
 
 /*
  * prune_append_rel_partitions
@@ -192,6 +164,9 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.boundinfo = rel->boundinfo;
 		context.has_default_part = rel->has_default_part;
 		context.partition_qual = rel->partition_qual;
+		context.prmlist = NULL;
+		context.econtext = NULL;
+		context.paramids = NULL;
 
 		/* process clauses; context.clauseinfo will be set */
 		generate_partition_clauses(&context, clauses);
@@ -558,6 +533,10 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				pc->inputcollid = opclause->inputcollid;
 				pc->value = valueexpr;
 
+				if (IsA(valueexpr, Param))
+					context->paramids = bms_add_member(context->paramids,
+											((Param *) valueexpr)->paramid);
+
 				/*
 				 * We don't turn a <> operator clause into a key right away.
 				 * Instead, the caller will hand over such clauses to
@@ -719,6 +698,11 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 														 leftop, rightop,
 														 InvalidOid,
 														 saop_coll);
+
+					if (IsA(rightop, Param))
+						context->paramids = bms_add_member(context->paramids,
+												((Param *) rightop)->paramid);
+
 					elem_clauses = lappend(elem_clauses, elem_clause);
 				}
 
@@ -969,7 +953,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				if (hash_clause == NULL)
 					hash_clause = pc;
 				/* check if another clause would contradict the one we have */
-				else if (partition_cmp_args(context->parttypid[i],
+				else if (partition_cmp_args(context,
+											context->parttypid[i],
 											context->partopfamily[i],
 											pc, pc, hash_clause,
 											&test_result))
@@ -1026,7 +1011,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * then because 7 < 5 is false, we leave a < 5 where it is and
 				 * effectively discard a < 7 as being redundant.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   pc, pc, btree_clauses[s],
 									   &test_result))
@@ -1083,7 +1069,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * eq clause is a = 3, then because 3 < 5, we no longer need
 				 * a < 5, because a = 3 is more restrictive.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   chk, eq, chk,
 									   &test_result))
@@ -1114,7 +1101,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *lt = btree_clauses[BTLessStrategyNumber - 1],
 					   *le = btree_clauses[BTLessEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   le, lt, le,
 								   &test_result))
@@ -1133,7 +1121,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *gt = btree_clauses[BTGreaterStrategyNumber - 1],
 					   *ge = btree_clauses[BTGreaterEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   ge, gt, ge,
 								   &test_result))
@@ -1171,9 +1160,9 @@ remove_redundant_clauses(PartitionPruneContext *context,
  * incompatible with the operator.
  */
 static bool
-partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result)
+partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result)
 {
 	Datum	left_value;
 	Datum	right_value;
@@ -1184,10 +1173,12 @@ partition_cmp_args(Oid parttypid, Oid partopfamily,
 	 * Try to extract an actual value from each arg.  This may fail if the
 	 * value is unknown in this context, in which case we cannot compare.
 	 */
-	if (!partkey_datum_from_expr(parttypid, leftarg->value, &left_value))
+	if (!partkey_datum_from_expr(context, parttypid, leftarg->value,
+		&left_value))
 		return false;
 
-	if (!partkey_datum_from_expr(parttypid, rightarg->value, &right_value))
+	if (!partkey_datum_from_expr(context, parttypid, rightarg->value,
+		&right_value))
 		return false;
 
 	/*
@@ -1310,12 +1301,16 @@ extract_bounding_datums(PartitionPruneContext *context,
 				case PART_OP_EQUAL:
 					Assert(incl);
 					if (need_next_eq &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1323,7 +1318,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 					}
 
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1333,7 +1330,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_LESS:
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1345,7 +1344,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_GREATER:
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1390,8 +1391,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 			PartClause *pc = (PartClause *) lfirst(lc);
 			Datum	datum;
 
-			if (partkey_datum_from_expr(context->parttypid[0], pc->value,
-										&datum))
+			if (partkey_datum_from_expr(context, context->parttypid[0],
+										pc->value, &datum))
 				keys->ne_datums[i++] = datum;
 		}
 		keys->n_ne_datums = i;
@@ -1471,7 +1472,8 @@ partition_op_strategy(char part_strategy, PartClause *pc, bool *incl)
  * set.  True is returned otherwise.
  */
 static bool
-partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -1509,11 +1511,180 @@ partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
 	 * Add more expression types here as needed to support the requirements
 	 * of the higher-level code.
 	 */
-	if (IsA(expr, Const))
+	switch (nodeTag(expr))
 	{
-		*value = ((Const *) expr)->constvalue;
-		return true;
+		case T_Const:
+			*value = ((Const *) expr)->constvalue;
+			return true;
+
+		case T_Param:
+			switch (((Param *) expr)->paramkind)
+			{
+				case PARAM_EXTERN:
+					if (context->prmlist)
+					{
+						Node	   *node;
+						Param	   *param = (Param *) expr;
+						ParamListInfo prmlist = context->prmlist;
+
+						node = eval_const_expressions_from_list(prmlist,
+																(Node *) param);
+						if (IsA(node, Const))
+						{
+							*value = ((Const *) node)->constvalue;
+							return true;
+						}
+					}
+
+				case PARAM_EXEC:
+					if (context->econtext)
+					{
+						Param	   *param = (Param *) expr;
+						ParamExecData *prm;
+						ExprContext *econtext = context->econtext;
+
+						prm = &(econtext->ecxt_param_exec_vals[param->paramid]);
+						if (unlikely(prm->execPlan != NULL))
+						{
+							ExecSetParamPlan((SubPlanState *) prm->execPlan,
+											 econtext);
+							Assert(prm->execPlan == NULL);
+						}
+						*value = prm->value;
+						return true;
+					}
+			}
 	}
 
 	return false;
 }
+
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes.  This is required in order to allow
+ *		us to perform any further partition pruning during execution.
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	check_stack_depth();
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->relid = rel->relid;
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->allsubnodes = NULL;
+	pinfo->nparts = nparts;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/* Determine the element in part_rel which belongs to this subpath. */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
+	}
+
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			List *subprunequal;
+
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			subprunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																subprunequal);
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
+	}
+
+	return pinfo;
+}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fe3b458..448c05a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,36 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time.  Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths.  We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3596,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index ed27ca9..27e4530 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -42,6 +43,43 @@ typedef struct PartitionDescData
 
 typedef struct PartitionDescData *PartitionDesc;
 
+/*
+ * Stores clauses which were matched to a partition key.
+ *
+ * Each matching "operator" clause is stored in the 'keyclauses' list for the
+ * partition key that it was matched to, except if the operator is <>, in
+ * which case, the clause is added to the 'ne_clauses' list.
+ *
+ * Boolean OR clauses whose at least one argument clause matches a partition
+ * key are added to the 'or_clauses' list.
+ *
+ * Based on a IS NULL or IS NOT NULL clause that was matched to a partition
+ * key, the corresponding bit in 'keyisnull' or 'keyisnotnull' is set.  A bit
+ * in 'keyisnotnull' may also be set when a strict OpExpr is encountered for
+ * the given partition key.
+ */
+typedef struct PartitionClauseInfo
+{
+	/* Lists of clauses indexed by the partition key */
+	List   *keyclauses[PARTITION_MAX_KEYS];
+
+	/* Each members is a List itself of a given OR clauses's arguments. */
+	List   *or_clauses;
+
+	/* List of clauses containing <> operator. */
+	List   *ne_clauses;
+
+	/* Nth (0 <= N < partnatts) bit set if the key is NULL or NOT NULL. */
+	Bitmapset   *keyisnull;
+	Bitmapset   *keyisnotnull;
+
+	/* True if at least one of above fields contains valid information. */
+	bool	foundkeyclauses;
+
+	/* True if mutually contradictory clauses were found. */
+	bool	constfalse;
+} PartitionClauseInfo;
+
 typedef struct PartitionPruneContext
 {
 	/* Table's range table index */
@@ -69,7 +107,13 @@ typedef struct PartitionPruneContext
 	PartitionBoundInfo	boundinfo;
 
 	/* Information about matched clauses */
-	struct PartitionClauseInfo *clauseinfo;
+	PartitionClauseInfo *clauseinfo;
+
+	ParamListInfo prmlist;
+	ExprContext *econtext;
+
+	/* ParamIds of clauses being used to determine partitions */
+	Bitmapset *paramids;
 } PartitionPruneContext;
 
 /*
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920..b6f1a9e 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index a953820..7db3a79 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1010,11 +1010,14 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
 struct AppendState;
 typedef struct AppendState AppendState;
+struct PartitionPruneContextCache;
+typedef struct PartitionPruneContextCache PartitionPruneContextCache;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
 
@@ -1026,6 +1029,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	PartitionPruneContextCache *contextcache; /* cache of prune contexts */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c097da6..d693f37 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f2e19ea..2264d54 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -250,6 +250,13 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition index into subplan indexes.  This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..ffb6daf 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,27 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into an Append node's subplan index.  This structure is used to
+ * recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	int			relid;		/* relation index of parent partition rel */
+	Oid			parentoid;	/* Oid of parent partition rel */
+	List	   *prunequal;	/* qual list for pruning partitions */
+	Bitmapset  *allsubnodes;	/* All subnode indexes at this level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 5579940..4d17f9a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1294,6 +1294,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 2b84ed9..6ca7558 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -22,4 +22,9 @@ extern void generate_partition_clauses(PartitionPruneContext *context,
 							List *clauses);
 extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context);
 
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index ef7173f..bde1858 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 948cad4..3ad17e1 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1475,3 +1475,1068 @@ explain (costs off) select * from like_op_noprune where a like '%BC';
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 08fc2db..a6fd74e 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -253,3 +253,298 @@ create table like_op_noprune2 partition of like_op_noprune for values in ('BCD')
 explain (costs off) select * from like_op_noprune where a like '%BC';
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
#77Rajkumar Raghuwanshi
rajkumar.raghuwanshi@enterprisedb.com
In reply to: David Rowley (#76)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Feb 21, 2018 at 2:36 PM, David Rowley <david.rowley@2ndquadrant.com>
wrote:

I've attached v11 of the patch.

Hi,

I have applied attached patch on head
"6f1d723b6359507ef55a81617167507bc25e3e2b" over Amit's v30 patches. while
testing further I got a server crash with below test case. Please take a
look.

CREATE TABLE prt1 (a int, b int, c varchar) PARTITION BY RANGE(a);
CREATE TABLE prt1_p1 PARTITION OF prt1 FOR VALUES FROM (0) TO (250);
CREATE TABLE prt1_p3 PARTITION OF prt1 FOR VALUES FROM (500) TO (600);
CREATE TABLE prt1_p2 PARTITION OF prt1 FOR VALUES FROM (250) TO (500);
INSERT INTO prt1 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0,
599, 2) i;
CREATE INDEX iprt1_p1_a on prt1_p1(a);
CREATE INDEX iprt1_p2_a on prt1_p2(a);
CREATE INDEX iprt1_p3_a on prt1_p3(a);
ANALYZE prt1;

CREATE TABLE prt2 (a int, b int, c varchar) PARTITION BY RANGE(b);
CREATE TABLE prt2_p1 PARTITION OF prt2 FOR VALUES FROM (0) TO (250);
CREATE TABLE prt2_p2 PARTITION OF prt2 FOR VALUES FROM (250) TO (500);
CREATE TABLE prt2_p3 PARTITION OF prt2 FOR VALUES FROM (500) TO (600);
INSERT INTO prt2 SELECT i, i, to_char(i, 'FM0000') FROM generate_series(0,
599, 3) i;
CREATE INDEX iprt2_p1_b on prt2_p1(b);
CREATE INDEX iprt2_p2_b on prt2_p2(b);
CREATE INDEX iprt2_p3_b on prt2_p3(b);
ANALYZE prt2;

CREATE TABLE plt1 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt1_p1 PARTITION OF plt1 FOR VALUES IN ('0000', '0003',
'0004', '0010');
CREATE TABLE plt1_p2 PARTITION OF plt1 FOR VALUES IN ('0001', '0005',
'0002', '0009');
CREATE TABLE plt1_p3 PARTITION OF plt1 FOR VALUES IN ('0006', '0007',
'0008', '0011');
INSERT INTO plt1 SELECT i, i, to_char(i/50, 'FM0000') FROM
generate_series(0, 599, 2) i;
CREATE INDEX iplt1_p1_c on plt1_p1(c);
CREATE INDEX iplt1_p2_c on plt1_p2(c);
CREATE INDEX iplt1_p3_c on plt1_p3(c);
ANALYZE plt1;

CREATE TABLE plt2 (a int, b int, c text) PARTITION BY LIST(c);
CREATE TABLE plt2_p1 PARTITION OF plt2 FOR VALUES IN ('0000', '0003',
'0004', '0010');
CREATE TABLE plt2_p2 PARTITION OF plt2 FOR VALUES IN ('0001', '0005',
'0002', '0009');
CREATE TABLE plt2_p3 PARTITION OF plt2 FOR VALUES IN ('0006', '0007',
'0008', '0011');
INSERT INTO plt2 SELECT i, i, to_char(i/50, 'FM0000') FROM
generate_series(0, 599, 3) i;
CREATE INDEX iplt2_p1_c on plt2_p1(c);
CREATE INDEX iplt2_p2_c on plt2_p2(c);
CREATE INDEX iplt2_p3_c on plt2_p3(c);
ANALYZE plt2;

select count(*) from prt1 x where (x.a,x.b) in (select t1.a,t2.b from prt1
t1,prt2 t2 where t1.a=t2.b)
and (x.c) in (select t3.c from plt1 t3,plt2 t4 where t3.c=t4.c);

/*
postgres=# select count(*) from prt1 x where (x.a,x.b) in (select t1.a,t2.b
from prt1 t1,prt2 t2 where t1.a=t2.b)
postgres-# and (x.c) in (select t3.c from plt1 t3,plt2 t4 where t3.c=t4.c);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.
*/

stack-trace give below :

/*
(gdb) bt
#0 0x00000000006ce6dc in ExecEvalParamExec (state=0x26e9ee0, op=0x26e9f78,
econtext=0x26ea390) at execExprInterp.c:2222
#1 0x00000000006cc66a in ExecInterpExpr (state=0x26e9ee0,
econtext=0x26ea390, isnull=0x7ffe0f75d77f "") at execExprInterp.c:1024
#2 0x00000000006cdd8c in ExecInterpExprStillValid (state=0x26e9ee0,
econtext=0x26ea390, isNull=0x7ffe0f75d77f "") at execExprInterp.c:1819
#3 0x00000000007db078 in ExecEvalExprSwitchContext (state=0x26e9ee0,
econtext=0x26ea390, isNull=0x7ffe0f75d77f "") at
../../../../src/include/executor/executor.h:305
#4 0x00000000007e2072 in evaluate_expr (expr=0x26a3cb0, result_type=25,
result_typmod=-1, result_collation=0) at clauses.c:4890
#5 0x00000000007e588a in partkey_datum_from_expr (context=0x26d3180,
parttypid=25, expr=0x26a3cb0, value=0x7ffe0f75da00) at partprune.c:1504
#6 0x00000000007e5243 in extract_bounding_datums (context=0x26d3180,
minimalclauses=0x7ffe0f75d900, keys=0x7ffe0f75da00) at partprune.c:1307
#7 0x00000000007e377d in get_partitions_from_clauses (context=0x26d3180)
at partprune.c:273
#8 0x00000000006ea2ec in set_valid_runtime_subplans_recurse
(node=0x269bf90, pinfo=0x7f6cf6765cf0, ctxcache=0x26d3158,
validsubplans=0x7ffe0f75de10) at nodeAppend.c:771
#9 0x00000000006e9ebf in set_valid_runtime_subplans (node=0x269bf90) at
nodeAppend.c:640
#10 0x00000000006e99b5 in choose_next_subplan_locally (node=0x269bf90) at
nodeAppend.c:426
#11 0x00000000006e9598 in ExecAppend (pstate=0x269bf90) at nodeAppend.c:224
#12 0x00000000006deb3a in ExecProcNodeFirst (node=0x269bf90) at
execProcnode.c:446
#13 0x00000000006fb9ee in ExecProcNode (node=0x269bf90) at
../../../src/include/executor/executor.h:239
#14 0x00000000006fbcc4 in ExecHashJoinImpl (pstate=0x2697808, parallel=0
'\000') at nodeHashjoin.c:262
#15 0x00000000006fc3fd in ExecHashJoin (pstate=0x2697808) at
nodeHashjoin.c:565
#16 0x00000000006deb3a in ExecProcNodeFirst (node=0x2697808) at
execProcnode.c:446
#17 0x000000000070c376 in ExecProcNode (node=0x2697808) at
../../../src/include/executor/executor.h:239
#18 0x000000000070c70e in ExecNestLoop (pstate=0x262c0a0) at
nodeNestloop.c:160
#19 0x00000000006deb3a in ExecProcNodeFirst (node=0x262c0a0) at
execProcnode.c:446
#20 0x00000000006fb9ee in ExecProcNode (node=0x262c0a0) at
../../../src/include/executor/executor.h:239
#21 0x00000000006fbcc4 in ExecHashJoinImpl (pstate=0x262bec8, parallel=0
'\000') at nodeHashjoin.c:262
#22 0x00000000006fc3fd in ExecHashJoin (pstate=0x262bec8) at
nodeHashjoin.c:565
#23 0x00000000006deb3a in ExecProcNodeFirst (node=0x262bec8) at
execProcnode.c:446
#24 0x00000000006ea5bd in ExecProcNode (node=0x262bec8) at
../../../src/include/executor/executor.h:239
#25 0x00000000006eaab0 in fetch_input_tuple (aggstate=0x262ba18) at
nodeAgg.c:406
#26 0x00000000006ecd40 in agg_retrieve_direct (aggstate=0x262ba18) at
nodeAgg.c:1736
#27 0x00000000006ec932 in ExecAgg (pstate=0x262ba18) at nodeAgg.c:1551
#28 0x00000000006deb3a in ExecProcNodeFirst (node=0x262ba18) at
execProcnode.c:446
#29 0x00000000006d59cd in ExecProcNode (node=0x262ba18) at
../../../src/include/executor/executor.h:239
#30 0x00000000006d8326 in ExecutePlan (estate=0x262b7c8,
planstate=0x262ba18, use_parallel_mode=0 '\000', operation=CMD_SELECT,
sendTuples=1 '\001', numberTuples=0,
direction=ForwardScanDirection, dest=0x7f6cf676c7f0, execute_once=1
'\001') at execMain.c:1721
#31 0x00000000006d5f9f in standard_ExecutorRun (queryDesc=0x258aa98,
direction=ForwardScanDirection, count=0, execute_once=1 '\001') at
execMain.c:361
#32 0x00000000006d5dbb in ExecutorRun (queryDesc=0x258aa98,
direction=ForwardScanDirection, count=0, execute_once=1 '\001') at
execMain.c:304
#33 0x00000000008b588b in PortalRunSelect (portal=0x25caa58, forward=1
'\001', count=0, dest=0x7f6cf676c7f0) at pquery.c:932
#34 0x00000000008b5519 in PortalRun (portal=0x25caa58,
count=9223372036854775807, isTopLevel=1 '\001', run_once=1 '\001',
dest=0x7f6cf676c7f0, altdest=0x7f6cf676c7f0,
completionTag=0x7ffe0f75e5e0 "") at pquery.c:773
#35 0x00000000008af540 in exec_simple_query (
query_string=0x2565728 "select count(*) from prt1 x where (x.a,x.b) in
(select t1.a,t2.b from prt1 t1,prt2 t2 where t1.a=t2.b) \nand (x.c) in
(select t3.c from plt1 t3,plt2 t4 where t3.c=t4.c);") at postgres.c:1120
#36 0x00000000008b37d4 in PostgresMain (argc=1, argv=0x25910e0,
dbname=0x2590f40 "postgres", username=0x2562228 "edb") at postgres.c:4144
#37 0x0000000000812afa in BackendRun (port=0x2588ea0) at postmaster.c:4412
#38 0x000000000081226e in BackendStartup (port=0x2588ea0) at
postmaster.c:4084
*/

Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation

#78David Rowley
david.rowley@2ndquadrant.com
In reply to: Rajkumar Raghuwanshi (#77)
Re: [HACKERS] Runtime Partition Pruning

On 21 February 2018 at 22:45, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:

select count(*) from prt1 x where (x.a,x.b) in (select t1.a,t2.b from prt1
t1,prt2 t2 where t1.a=t2.b)
and (x.c) in (select t3.c from plt1 t3,plt2 t4 where t3.c=t4.c);

Thanks for the test case.

It seems like the x.c Param for some reason has a ParamId of 0. I need
to go learn a bit more about Params to understand why this is.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#79Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#76)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/02/21 18:06, David Rowley wrote:

Other things I don't particularly like about the current patch are how
I had to construct the partition key expressions in
set_valid_runtime_subplans_recurse(). This pretty much uses code
borrowed from set_baserel_partition_key_exprs(). One way around that
would be to change the partprune.c code to deal with the
partkey->partattrs and consume an expression from the list on attnum =
0. I didn't do that as I want to minimise how much I touch Amit's
patch before it gets committed as doing so would likely just cause
more headaches for me keeping this merged with his work. Another
option to resolve this would be to put the partition key expressions
into the PartitionPruneInfo.

Another way could be to refactor the code you've borrowed from
set_baserel_partition_key_exprs() into its own function that is exported
by some optimizer header.

I removed PartitionKey/Relation from signatures of various functions of my
patch to avoid having to re-heap_open() the relation per [1]/messages/by-id/CA+Tgmoabi-29Vs8H0xkjtYB=cU+GVCrNwPz7okpa3KsoLmdEUQ@mail.gmail.com.

I've attached v11 of the patch.

Some comments:

* I noticed that the patch adds a function to bitmapset.c which you might
want to extract into its own patch, like your patch to add bms_add_range()
that got committed as 84940644d [2]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=84940644d.

* Maybe it's better to add `default: break;` in the two switch's
you've added in partkey_datum_from_expr()

partprune.c: In function ‘partkey_datum_from_expr’:
partprune.c:1555:2: warning: enumeration value ‘T_Invalid’ not handled in
switch [-Wswitch]
switch (nodeTag(expr))

partprune.c:1562:4: warning: enumeration value ‘PARAM_SUBLINK’ not handled
in switch [-Wswitch]
switch (((Param *) expr)->paramkind)

* I see that you moved PartitionClauseInfo's definition from partprune.c
to partition.h. Isn't it better to move it to partprune.h?

Thanks,
Amit

[1]: /messages/by-id/CA+Tgmoabi-29Vs8H0xkjtYB=cU+GVCrNwPz7okpa3KsoLmdEUQ@mail.gmail.com
/messages/by-id/CA+Tgmoabi-29Vs8H0xkjtYB=cU+GVCrNwPz7okpa3KsoLmdEUQ@mail.gmail.com

[2]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=84940644d

#80David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#79)
Re: [HACKERS] Runtime Partition Pruning

On 22 February 2018 at 22:31, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Some comments:

Hi Amit,

Thanks for looking at this. I'll work through your comments and
produce a patch sometime in the near future.

One problem that I'm facing now is down to the way I'm gathering the
ParamIds that match the partkeys. As you'll see from the patch I've
added a 'paramids' field to PartitionPruneContext and I'm populating
this when the clauses are being pre-processed in
extract_partition_clauses(). The problem is that the or_clauses are
not pre-processed at all, so the current patch will not properly
perform run-time pruning when the Params are "hidden" in OR branches.

One way I thought to fix this was to change the clause processing to
create an array of PartitionClauseInfos, one for each OR branch. This
would also improve the performance of the run-time pruning, meaning
that all of the or clauses would be already matched to the partition
keys once, rather than having to redo that again each time a Param
changes its value.

If I go and write a patch to do that, would you want it in your patch,
or would you rather I kept it over here? Or perhaps you have a better
idea on how to solve...?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#81Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#76)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 02/21/2018 04:06 AM, David Rowley wrote:

I've attached v11 of the patch.

Are UPDATE and DELETE suppose to be supported ?

With

-- test.sql --
CREATE TABLE test (a integer NOT NULL, b integer) PARTITION BY HASH(a);
CREATE TABLE test_p00 PARTITION OF test FOR VALUES WITH (MODULUS 2,
REMAINDER 0);
CREATE TABLE test_p01 PARTITION OF test FOR VALUES WITH (MODULUS 2,
REMAINDER 1);
CREATE INDEX idx_test_a ON test (a);
CREATE INDEX idx_test_b ON test (b);

INSERT INTO test (SELECT i,i FROM generate_series(1, 1000000) AS i);

ANALYZE;
-- test.sql --

and

UPDATE test SET b = 1 WHERE a = ?
DELETE FROM test WHERE a = ?

both shows that all partitions are scanned;

Update on test
Update on test_p00
Update on test_p01
-> Index Scan using test_p00_a_idx on test_p00
Index Cond: (a = 1)
-> Index Scan using test_p01_a_idx on test_p01
Index Cond: (a = 1)

Using prune_v32 and runtime_v11 with conflicts resolved.

Best regards,
Jesper

#82David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#80)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 23 February 2018 at 01:15, David Rowley <david.rowley@2ndquadrant.com> wrote:

One problem that I'm facing now is down to the way I'm gathering the
ParamIds that match the partkeys. As you'll see from the patch I've
added a 'paramids' field to PartitionPruneContext and I'm populating
this when the clauses are being pre-processed in
extract_partition_clauses(). The problem is that the or_clauses are
not pre-processed at all, so the current patch will not properly
perform run-time pruning when the Params are "hidden" in OR branches.

One way I thought to fix this was to change the clause processing to
create an array of PartitionClauseInfos, one for each OR branch. This
would also improve the performance of the run-time pruning, meaning
that all of the or clauses would be already matched to the partition
keys once, rather than having to redo that again each time a Param
changes its value.

If I go and write a patch to do that, would you want it in your patch,
or would you rather I kept it over here? Or perhaps you have a better
idea on how to solve...?

Hi Amit,

I've attached a patch which does this. For now, the patch is only
intended to assist in the discussion of the above idea.

The patch is based on a WIP version of run-time pruning that I'm not
quite ready to post yet, but with a small amount of work you could
probably take it and base it on your faster partition pruning v31
patch [1]/messages/by-id/00ae2273-bb6b-1287-9ebc-5459b37c9078@lab.ntt.co.jp.

I ended up pulling the PartitionPruneInfo out of the
PartitionPruneContext. This was required due how I've now made
extract_partition_clauses() recursively call itself. We don't want to
overwrite the context's clauseinfo with the one from the recursive
call. A side effect of this is that the subcontext is no longer
required when processing the OR clauses. You only did this so that the
context's clauseinfo was not overwritten. I also think it's better to
seperate out the inputs and outputs. Anything in context was more
intended to be for input fields.

Let me know your thoughts about this. If you don't want it for faster
partition pruning, then I'll probably go and tidy it up and include it
for run-time pruning.

[1]: /messages/by-id/00ae2273-bb6b-1287-9ebc-5459b37c9078@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

generate_PartitionClauseInfos_for_or_clauses.patchapplication/octet-stream; name=generate_PartitionClauseInfos_for_or_clauses.patchDownload
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7171682..7658906 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -83,6 +83,7 @@ struct ParallelAppendState
 struct PartitionPruneContextCache
 {
 	PartitionPruneContext *context;
+	PartitionClauseInfo *partclauseinfo;
 	PartitionPruneContextCache *subcache;
 };
 
@@ -714,7 +715,7 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 		context->econtext = node->ps.ps_ExprContext;
 		context->paramids = NULL;
 
-		generate_partition_clauses(context, pinfo->prunequal);
+		ctxcache->partclauseinfo = generate_partition_clauses(context, pinfo->prunequal);
 
 		node->part_prune_params = bms_add_members(node->part_prune_params,
 												  context->paramids);
@@ -730,7 +731,7 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 	 * Detect if any impossibilities were discovered during
 	 * generate_partition_clauses
 	 */
-	if (context->clauseinfo->constfalse)
+	if (ctxcache->partclauseinfo->constfalse)
 	{
 		bms_free(*validsubplans);
 		*validsubplans = NULL;
@@ -747,7 +748,8 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 	 * params.
 	 */
 	if (!bms_is_empty(context->paramids))
-		partset = get_partitions_from_clauses(context);
+		partset = get_partitions_from_clauses(context,
+											  ctxcache->partclauseinfo);
 	else
 		partset = pinfo->allsubnodes;
 
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 81f66bf..010333e 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -24,21 +24,19 @@
  * pruning from a list of clauses containing clauses that reference a given
  * partitioned table.  For example, prune_append_rel_partitions() calls this
  * function, because a partitioned table's rel->baserestrictinfo may contain
- * clauses that might be useful for partitioning.  Caller must have set up a
- * valid partition pruning context in the form of struct PartitionPruneContext,
- * that is, each of its fields other other than clauseinfo must be valid before
- * calling here.  After extracting relevant clauses, clauseinfo is filled with
- * information that will be used for actual pruning.
+ * clauses that might be useful for partitioning.  The list of clauses is
+ * processed and a PartitionClauseInfo is returned which contains details of
+ * any clauses which could be matched to the partition keys of the relation
+ * defined in the context.
  *
  * get_partitions_from_clauses()
  *
- * This is to be called to prune partitions based on relevant partitioning
- * clauses.  Caller must have called generate_partition_clauses() at least
- * once and hence a valid partition pruning context must have already been
- * created.  Especially, PartitionPruneContext.clauseinfo must contain valid
- * information.  Partition pruning proceeds by extracting constant values
- * from the clauses and comparing it with the partition bounds while also
- * taking into account strategies of the operators in the matched clauses.
+ * This is to be called to prune partitions based on 'partclauseinfo'.  Caller
+ * must have called generate_partition_clauses() in order to have generated
+ * a valid PartitionClauseInfo.  Partition pruning proceeds by extracting
+ * constant values from the clauses and comparing it with the partition bounds
+ * while also taking into account strategies of the operators in the matched
+ * clauses.
  *
  * make_partition_pruneinfo()
  *
@@ -104,18 +102,22 @@ typedef enum PartOpStrategy
 	PART_OP_GREATER
 } PartOpStrategy;
 
-static void extract_partition_clauses(PartitionPruneContext *context,
-						   List *clauses);
+static PartitionClauseInfo *extract_partition_clauses(
+						  PartitionPruneContext *context,
+						  List *clauses);
 static bool match_boolean_partition_clause(Expr *clause, Expr *partkey,
 							   Expr **rightop);
 static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
-								   List *or_args);
+							List *or_args,
+							List *or_partclauselist);
 static void remove_redundant_clauses(PartitionPruneContext *context,
+						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses);
 static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
 				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
 				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
+						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
@@ -144,6 +146,7 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 	else
 	{
 		PartitionPruneContext context;
+		PartitionClauseInfo *partclauseinfo;
 		int		partnatts = rel->part_scheme->partnatts,
 				i;
 
@@ -168,13 +171,16 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.econtext = NULL;
 		context.paramids = NULL;
 
-		/* process clauses; context.clauseinfo will be set */
-		generate_partition_clauses(&context, clauses);
+		/* process clauses */
+		partclauseinfo = generate_partition_clauses(&context, clauses);
 
-		if (!context.clauseinfo->constfalse)
+		if (!partclauseinfo->constfalse)
 		{
 			/* Actual pruning happens here. */
-			Bitmapset *partindexes = get_partitions_from_clauses(&context);
+			Bitmapset *partindexes;
+
+			partindexes = get_partitions_from_clauses(&context,
+													  partclauseinfo);
 
 			/* Add selected partitions' RT indexes to result. */
 			i = -1;
@@ -188,13 +194,10 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 
 /*
  * generate_partition_clauses
- *		Analyzes clauses to find those that match the partition key and sets
- *		context->clauseinfo
- *
- * Ideally, this should be called only once for a given query and a given
- * partitioned table.
+ *		Processes 'clauses' and returns a PartitionClauseInfo which contains
+ *		the details of any clauses which were matched to partition keys.
  */
-void
+PartitionClauseInfo *
 generate_partition_clauses(PartitionPruneContext *context, List *clauses)
 {
 	/* The clauses list may be modified below, so better make a copy. */
@@ -227,25 +230,26 @@ generate_partition_clauses(PartitionPruneContext *context, List *clauses)
 		clauses = list_concat(clauses, partqual);
 	}
 
-	/* And away we go to do the real work; context->clauseinfo will be set */
-	extract_partition_clauses(context, clauses);
+	/* pre-process the clauses and generate the PartitionClauseInfo */
+	return extract_partition_clauses(context, clauses);
 }
 
 /*
  * get_partitions_from_clauses
  *		Determine partitions that could possible contain a record that
- *		satisfies clauses as described in context->clauseinfo
+ *		satisfies clauses as described in partclauseinfo
  *
  * Returns a Bitmapset of the matching partition indexes, or NULL if none can
  * match.
  */
 Bitmapset *
-get_partitions_from_clauses(PartitionPruneContext *context)
+get_partitions_from_clauses(PartitionPruneContext *context,
+							PartitionClauseInfo *partclauseinfo)
 {
-	PartitionClauseInfo	*partclauseinfo = context->clauseinfo;
 	PartScanKeyInfo		keys;
 	Bitmapset 		   *result;
-	ListCell *lc;
+	ListCell		   *lc;
+	ListCell		   *lc2;
 
 	Assert(partclauseinfo != NULL);
 	Assert(!partclauseinfo->constfalse);
@@ -260,17 +264,18 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 		List *minimalclauses[PARTITION_MAX_KEYS];
 
 		/*
-		 * For each partition key column, populate its slot in minimalclauses
-		 * with the most restrictive of the clauses from the corresponding
-		 * list in context->clauseinfo.
+		 * For each partition key column, populate its element in
+		 * minimalclauses with the most restrictive set of the clauses from
+		 * the corresponding partition key in partclauseinfo.
 		 */
-		remove_redundant_clauses(context, minimalclauses);
+		remove_redundant_clauses(context, partclauseinfo, minimalclauses);
 
 		/* Did remove_redundant_clauses find any contradicting clauses? */
 		if (partclauseinfo->constfalse)
 			return NULL;
 
-		if (extract_bounding_datums(context, minimalclauses, &keys))
+		if (extract_bounding_datums(context, partclauseinfo, minimalclauses,
+			&keys))
 		{
 			result = get_partitions_for_keys(context, &keys);
 
@@ -292,12 +297,14 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 	}
 
 	/* Now apply the OR clauses. */
-	foreach(lc, partclauseinfo->or_clauses)
+	forboth(lc, partclauseinfo->or_clauses, lc2,
+			partclauseinfo->or_partclauseinfos)
 	{
 		List *or_args = (List *) lfirst(lc);
+		List *or_partclauselist = lfirst(lc2);
 		Bitmapset *or_parts;
 
-		or_parts = get_partitions_from_or_args(context, or_args);
+		or_parts = get_partitions_from_or_args(context, or_args, or_partclauselist);
 
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -327,26 +334,27 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 /*
  * extract_partition_clauses
  *		Processes 'clauses' to extract clause matching the partition key.
- *		This adds matched clauses to the list corresponding to particular key
- *		in context->clauseinfo.  Also collects other useful clauses to assist
- *		in partition elimination, such as OR clauses, clauses containing <>
- *		operator, and IS [NOT] NULL clauses
+ *		Returns a PartitionClauseInfo which stores the clauses which were
+ *		matched to the partition key.  The PartitionClauseInfo also collects
+ *		other useful clauses to assist in partition elimination, such as OR
+ *		clauses, clauses containing <> operator, and IS [NOT] NULL clauses
  *
  * We may also discover some contradiction in the clauses which means that no
- * partition can possibly match.  In this case, the function sets
- * context->clauseinfo's 'constfalse' to true and exits immediately without
- * processing any further clauses.  In this case, the caller must be careful
- * not to assume the context->clauseinfo is fully populated with all clauses.
+ * partition can possibly match.  In this case, the function sets the
+ * returned PartitionClauseInfo's 'constfalse' to true and exits immediately
+ * without processing any further clauses.  In this case, the caller must be
+ * careful not to assume the return value is fully populated with all clauses.
  */
-static void
+static PartitionClauseInfo *
 extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 {
 	PartitionClauseInfo *partclauseinfo;
 	ListCell   *lc;
 
-	context->clauseinfo = partclauseinfo = palloc(sizeof(PartitionClauseInfo));
+	partclauseinfo = palloc(sizeof(PartitionClauseInfo));
 	memset(partclauseinfo->keyclauses, 0, sizeof(partclauseinfo->keyclauses));
 	partclauseinfo->or_clauses = NIL;
+	partclauseinfo->or_partclauseinfos = NIL;
 	partclauseinfo->ne_clauses = NIL;
 	partclauseinfo->keyisnull = NULL;
 	partclauseinfo->keyisnotnull = NULL;
@@ -367,7 +375,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				!DatumGetBool(((Const *) clause)->constvalue))
 			{
 				partclauseinfo->constfalse = true;
-				return;
+				return partclauseinfo;
 			}
 		}
 
@@ -558,7 +566,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				if (bms_is_member(i, partclauseinfo->keyisnull))
 				{
 					partclauseinfo->constfalse = true;
-					return;
+					return partclauseinfo;
 				}
 				/* Record that a strict clause has been seen for this key */
 				partclauseinfo->keyisnotnull =
@@ -734,7 +742,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 						if (bms_is_member(i, partclauseinfo->keyisnotnull))
 						{
 							partclauseinfo->constfalse = true;
-							return;
+							return partclauseinfo;
 						}
 						partclauseinfo->keyisnull =
 									bms_add_member(partclauseinfo->keyisnull,
@@ -746,7 +754,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 						if (bms_is_member(i, partclauseinfo->keyisnull))
 						{
 							partclauseinfo->constfalse = true;
-							return;
+							return partclauseinfo;
 						}
 
 						partclauseinfo->keyisnotnull =
@@ -760,6 +768,31 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 			partclauseinfo->foundkeyclauses = true;
 		}
 	}
+
+	/*
+	 * Now pre-process any OR clauses found above and generate
+	 * PartitionClauseInfos for them.
+	 */
+	foreach(lc, partclauseinfo->or_clauses)
+	{
+		List *or_args = lfirst(lc);
+		List *pclauselist = NIL;
+		ListCell *lc2;
+
+		foreach (lc2, or_args)
+		{
+			List *clauses = list_make1(lfirst(lc2));
+			PartitionClauseInfo *orpartclauseinfo;
+
+			orpartclauseinfo = extract_partition_clauses(context, clauses);
+			pclauselist = lappend(pclauselist, orpartclauseinfo);
+		}
+
+		partclauseinfo->or_partclauseinfos =
+					lappend(partclauseinfo->or_partclauseinfos, pclauselist);
+	}
+
+	return partclauseinfo;
 }
 
 /*
@@ -829,10 +862,11 @@ match_boolean_partition_clause(Expr *clause, Expr *partkey,
  * clause in or_args.
  */
 static Bitmapset *
-get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
+get_partitions_from_or_args(PartitionPruneContext *context, List *or_args,
+							List *or_partclauselist)
 {
 	Bitmapset	   *result = NULL;
-	ListCell	   *lc;
+	ListCell	   *lc, *lc2;
 
 	/*
 	 * When matching an OR expression, it is only checked if at least one of
@@ -843,20 +877,13 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 	 * clause refutes its partition constraint, that is, we can eliminate all
 	 * of its partitions.
 	 */
-	foreach(lc, or_args)
+	forboth(lc, or_args, lc2, or_partclauselist)
 	{
 		List *clauses = list_make1(lfirst(lc));
-		PartitionPruneContext subcontext;
+		PartitionClauseInfo *or_pclause = lfirst(lc2);
 		Bitmapset *arg_partset;
 
-		/*
-		 * All fields except clauseinfo are same as in the parent context,
-		 * which will be set by calling extract_partition_clauses().
-		 */
-		memcpy(&subcontext, context, sizeof(PartitionPruneContext));
-		extract_partition_clauses(&subcontext, clauses);
-
-		if (!subcontext.clauseinfo->foundkeyclauses)
+		if (!or_pclause->foundkeyclauses)
 		{
 			List *partconstr = context->partition_qual;
 
@@ -873,8 +900,8 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 			return bms_add_range(NULL, 0, context->nparts - 1);
 		}
 
-		if (!subcontext.clauseinfo->constfalse)
-			arg_partset = get_partitions_from_clauses(&subcontext);
+		if (!or_pclause->constfalse)
+			arg_partset = get_partitions_from_clauses(context, or_pclause);
 		else
 			arg_partset = NULL;
 
@@ -887,8 +914,8 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 
 /*
  * remove_redundant_clauses
- *		Processes the clauses contained in context->clauseinfo to remove the
- *		ones that are superseeded by other clauses which are more restrictive.
+ *		Process 'partpruneinfo' to remove the clauses that are superseeded
+ *		by other clauses which are more restrictive.
  *
  * Finished lists of clauses are returned in *minimalclauses which is an array
  * with one slot for each of the partition keys.
@@ -900,16 +927,16 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
  * that the clauses cannot possibly match any partition.  Impossible clauses
  * include things like: x = 1 AND x = 2, x > 0 and x < 10.  The function
  * returns right after finding such a clause and before returning, sets
- * constfalse in context->clauseinfo to inform the caller that we found such
+ * constfalse in 'partclauseinfo' to inform the caller that we found such
  * clause.
  */
 static void
 remove_redundant_clauses(PartitionPruneContext *context,
+						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses)
 {
 	PartClause *hash_clause,
 			   *btree_clauses[BTMaxStrategyNumber];
-	PartitionClauseInfo *partclauseinfo = context->clauseinfo;
 	ListCell *lc;
 	int		s;
 	int		i;
@@ -1220,8 +1247,9 @@ partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
 
 /*
  * extract_bounding_datums
- *		Process clauses in context->clauseinfo and populate 'keys' with all
- *		min/max/equal/not-equal values that we're able to determine.
+ *		Process 'clauseinfo' and populate 'keys' with all
+ *		min/max/equal/not-equal values that we're able to
+ *		determine.
  *
  * *minimalclauses is an array with partnatts members, each of which is a list
  * of the most restrictive clauses of each operator strategy for the given
@@ -1237,9 +1265,9 @@ partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
  */
 static bool
 extract_bounding_datums(PartitionPruneContext *context,
+						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys)
 {
-	PartitionClauseInfo *clauseinfo = context->clauseinfo;
 	bool		need_next_eq,
 				need_next_min,
 				need_next_max;
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index a761e65..ff68d4a 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -39,6 +39,9 @@ typedef struct PartitionClauseInfo
 	/* Each members is a List itself of a given OR clauses's arguments. */
 	List   *or_clauses;
 
+	/* each OR clause processed into a PartitionClauseInfo */
+	List *or_partclauseinfos;
+
 	/* List of clauses containing <> operator. */
 	List   *ne_clauses;
 
@@ -55,9 +58,11 @@ typedef struct PartitionClauseInfo
 
 extern Relids prune_append_rel_partitions(PlannerInfo *root,
 							RelOptInfo *rel);
-extern void generate_partition_clauses(PartitionPruneContext *context,
-							List *clauses);
-extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context);
+extern PartitionClauseInfo *generate_partition_clauses(
+						   PartitionPruneContext *context,
+						   List *clauses);
+extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context,
+							PartitionClauseInfo *partclauseinfo);
 
 extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
 						 RelOptInfo *rel,
#83Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#82)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/02/23 0:11, David Rowley wrote:

On 23 February 2018 at 01:15, David Rowley <david.rowley@2ndquadrant.com> wrote:

One problem that I'm facing now is down to the way I'm gathering the
ParamIds that match the partkeys. As you'll see from the patch I've
added a 'paramids' field to PartitionPruneContext and I'm populating
this when the clauses are being pre-processed in
extract_partition_clauses(). The problem is that the or_clauses are
not pre-processed at all, so the current patch will not properly
perform run-time pruning when the Params are "hidden" in OR branches.

One way I thought to fix this was to change the clause processing to
create an array of PartitionClauseInfos, one for each OR branch. This
would also improve the performance of the run-time pruning, meaning
that all of the or clauses would be already matched to the partition
keys once, rather than having to redo that again each time a Param
changes its value.

If I go and write a patch to do that, would you want it in your patch,
or would you rather I kept it over here? Or perhaps you have a better
idea on how to solve...?

I've attached a patch which does this. For now, the patch is only
intended to assist in the discussion of the above idea.

The patch is based on a WIP version of run-time pruning that I'm not
quite ready to post yet, but with a small amount of work you could
probably take it and base it on your faster partition pruning v31
patch [1].

I ended up pulling the PartitionPruneInfo out of the
PartitionPruneContext. This was required due how I've now made
extract_partition_clauses() recursively call itself. We don't want to
overwrite the context's clauseinfo with the one from the recursive
call. A side effect of this is that the subcontext is no longer
required when processing the OR clauses. You only did this so that the
context's clauseinfo was not overwritten. I also think it's better to
seperate out the inputs and outputs. Anything in context was more
intended to be for input fields.

Let me know your thoughts about this. If you don't want it for faster
partition pruning, then I'll probably go and tidy it up and include it
for run-time pruning.

Thanks for the patch.

I don't have time today to look at the patch closely, but at first blush,
it seems like something I should incorporate into my own patch, because
it's restructuring the partprune.c code. I will study the issue and your
proposed solution in the form of this restructuring more closely over the
weekend and reply (probably with a new version of my patch) on Monday.

Thanks,
Amit

#84David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#79)
3 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 22 February 2018 at 22:31, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Some comments:

* I noticed that the patch adds a function to bitmapset.c which you might
want to extract into its own patch, like your patch to add bms_add_range()
that got committed as 84940644d [2].

I've made that 0001 in the series

* Maybe it's better to add `default: break;` in the two switch's
you've added in partkey_datum_from_expr()

partprune.c: In function ‘partkey_datum_from_expr’:
partprune.c:1555:2: warning: enumeration value ‘T_Invalid’ not handled in
switch [-Wswitch]
switch (nodeTag(expr))

partprune.c:1562:4: warning: enumeration value ‘PARAM_SUBLINK’ not handled
in switch [-Wswitch]
switch (((Param *) expr)->paramkind)

I wasn't aware of that gcc flag. I was also surprised to see a clean
compile from master with it enabled. This area has been changed a bit
from the last patch, but the remaining switch now has a default:
return false;

* I see that you moved PartitionClauseInfo's definition from partprune.c
to partition.h. Isn't it better to move it to partprune.h?

Moved. I'd done it the other way to try to reduce the number of
planner headers included in the executor, but will defer to your
better judgement, as I see you're busy working on improving this area
in another patch set.

I've attached an updated set of patches.

I hope this also addresses Rajkumar reported crash. I ended up making
some changes to how the Param values are determined by reusing more of
the existing executor code rather than duplicating it in
partkey_datum_from_expr. I really could use a sanity check on my
changes to that function now, especially the cross type portion.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v12-0001-Add-bms_prev_member-function.patchapplication/octet-stream; name=v12-0001-Add-bms_prev_member-function.patchDownload
From 3569b5d82dee7bf5600846132aa7ccc0c14564c3 Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Fri, 23 Feb 2018 23:49:05 +1300
Subject: [PATCH v12 1/3] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a..3578c8f 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920..b6f1a9e 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
1.9.5.msysgit.1

v12-0002-Allow-partition-elimination-to-occur-during-exec.patchapplication/octet-stream; name=v12-0002-Allow-partition-elimination-to-occur-during-exec.patchDownload
From 46e5e803c9f9f4fa79f6c18d0bc7ca49c513192f Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Sat, 24 Feb 2018 00:11:04 +1300
Subject: [PATCH v12 2/3] Allow partition elimination to occur during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using information which is available during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to participate too.  We can also perform partition elimination in
the following cases:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Unlike the case of partition elimination during query planning, when the
elimination takes place during execution, the Append's subnode are still visible
in the EXPLAIN output.  In order to determine if pruning has actually taken
place the EXPLAIN ANALYZE must be viewed.  If a certain Append subplan was
never executed due to elimination of the partition then the execution timing
area will state "(never executed)".  Whereas, if, for example in the case of
parameterized nested loops, the number of loops stated in the EXPLAIN ANALYZE
output for certain subplans may appear lower than others due to the subplan
having been scanned fewer times.

For now, only the Append nodes when used in a SELECT query allows this
execution time elimination to take place.  The features added here are likely
easily extendable into MergeAppend and possibly also for DML statements too,
such as UPDATE and DELETE.
---
 src/backend/catalog/partition.c               |   50 ++
 src/backend/executor/nodeAppend.c             |  328 +++++++-
 src/backend/nodes/copyfuncs.c                 |   32 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   49 +-
 src/backend/optimizer/plan/planner.c          |    3 +-
 src/backend/optimizer/prep/prepunion.c        |    4 +-
 src/backend/optimizer/util/clauses.c          |   19 +
 src/backend/optimizer/util/partprune.c        |  312 +++++--
 src/backend/optimizer/util/pathnode.c         |   37 +-
 src/backend/optimizer/util/plancat.c          |   36 +-
 src/include/catalog/partition.h               |    9 +
 src/include/nodes/execnodes.h                 |    8 +
 src/include/nodes/nodes.h                     |    1 +
 src/include/nodes/plannodes.h                 |    7 +
 src/include/nodes/primnodes.h                 |   23 +
 src/include/nodes/relation.h                  |    2 +
 src/include/optimizer/clauses.h               |    3 +
 src/include/optimizer/partprune.h             |   42 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1115 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  312 +++++++
 23 files changed, 2260 insertions(+), 148 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 59e3234..7ddc409 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -4198,3 +4198,53 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * build_partition_expressions
+ *		Build a list of partition key expressions.  Plain attibute keys will
+ *		be build as Vars with the varno set to 'varno'.
+ */
+List **
+build_partition_expressions(PartitionKey partkey, Index varno)
+{
+	int			partnatts;
+	int			cnt;
+	List	  **partexprs;
+	ListCell   *lc;
+
+	partnatts = partkey->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * partnatts);
+	lc = list_head(partkey->partexprs);
+
+	for (cnt = 0; cnt < partnatts; cnt++)
+	{
+		Expr	   *partexpr;
+		AttrNumber	attno = partkey->partattrs[cnt];
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Assert(attno > 0);
+
+			partexpr = (Expr *) makeVar(varno, attno,
+										partkey->parttypid[cnt],
+										partkey->parttypmod[cnt],
+										partkey->parttypcoll[cnt], 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expression with given varno. */
+			partexpr = (Expr *) copyObject(lfirst(lc));
+			if (varno != 1)
+				ChangeVarNodes((Node *) partexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt] = list_make1(partexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7a3dd2e..7171682 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,10 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "optimizer/partprune.h"
+#include "optimizer/plancat.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -76,12 +80,25 @@ struct ParallelAppendState
 	bool		pa_finished[FLEXIBLE_ARRAY_MEMBER];
 };
 
+struct PartitionPruneContextCache
+{
+	PartitionPruneContext *context;
+	PartitionPruneContextCache *subcache;
+};
+
 #define INVALID_SUBPLAN_INDEX		-1
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +144,34 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = NULL; /* determined later */
+	appendstate->part_prune_info = node->part_prune_info;
+	appendstate->contextcache = NULL; /* populate this as needed below */
+
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
@@ -149,18 +194,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +292,17 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +322,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,22 +412,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
-
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -408,6 +473,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -460,6 +536,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +612,198 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	if (!node->contextcache)
+		node->contextcache = palloc0(sizeof(PartitionPruneContextCache));
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   node->contextcache,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context;
+	Bitmapset	   *partset;
+	int				i;
+
+	check_stack_depth();
+
+	/*
+	 * If the PartitionPruneContext has not yet been initialized for this rel
+	 * yet, then do that now.
+	 */
+	if (!ctxcache->context)
+	{
+		PartitionDesc	partdesc;
+		Relation		rel;
+		PartitionKey	partkey;
+		MemoryContext	oldContext;
+		List		  **partexprs;
+		int				partnatts;
+		int				i;
+
+		oldContext = MemoryContextSwitchTo(node->ps.state->es_query_cxt);
+
+		ctxcache->context = context = palloc(sizeof(PartitionPruneContext));
+		ctxcache->subcache = palloc0(sizeof(PartitionPruneContextCache) *
+									 pinfo->nparts);
+
+		rel = relation_open(pinfo->parentoid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->relid = pinfo->relid;
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+		partexprs = build_partition_expressions(partkey, pinfo->relid);
+		context->partkeys = (Expr **) palloc(sizeof(Expr *) * partnatts);
+
+		for (i = 0; i < partnatts; i++)
+			context->partkeys[i] = (Expr *) linitial(partexprs[i]);
+
+		context->parttypid = partkey->parttypid;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		if (OidIsValid(get_default_oid_from_partdesc(partdesc)))
+			context->has_default_part = true;
+		else
+			context->has_default_part = false;
+
+		context->partition_qual = RelationGetPartitionQual(rel);
+
+		context->planstate = &node->ps;
+		context->econtext = node->ps.ps_ExprContext;
+		context->paramids = NULL;
+
+		generate_partition_clauses(context, pinfo->prunequal);
+
+		node->part_prune_params = bms_add_members(node->part_prune_params,
+												  context->paramids);
+
+		relation_close(rel, NoLock);
+
+		MemoryContextSwitchTo(oldContext);
+	}
+	else
+		context = ctxcache->context;
+
+	/*
+	 * Detect if any impossibilities were discovered during
+	 * generate_partition_clauses
+	 */
+	if (context->clauseinfo->constfalse)
+	{
+		bms_free(*validsubplans);
+		*validsubplans = NULL;
+		return;
+	}
+
+	/*
+	 * 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 subpartitions as we may find their partitions keys match some
+	 * params.
+	 */
+	if (!bms_is_empty(context->paramids))
+		partset = get_partitions_from_clauses(context);
+	else
+		partset = pinfo->allsubnodes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node,
+												pinfo->subpartindex[i],
+												&ctxcache->subcache[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than what was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 1bb76dd..4e28144 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -244,6 +244,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2132,6 +2133,34 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(relid);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_BITMAPSET_FIELD(allsubnodes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts *
+					   sizeof(PartitionPruneInfo *));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			COPY_NODE_FIELD(subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5027,6 +5056,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 17eae10..fe3c1a6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1587,7 +1587,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1629,8 +1629,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1678,7 +1678,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1734,7 +1734,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2000,7 +2000,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3..2e289d4 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index da0cc7f..d82fee7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -204,7 +205,8 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1022,6 +1024,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1059,6 +1063,41 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist that could possibly be useful to use for
+		 * performing further partition pruning during execution, then
+		 * we'll generate a PartitionPruneInfo to store these quals and
+		 * allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1067,7 +1106,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5320,7 +5360,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5332,7 +5373,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 8fa90b1..2e96491 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3719,7 +3719,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f01119e..146e202 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -593,7 +593,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -705,7 +705,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 0c1f239..dccbcc1 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index f945402..21e86ff 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -40,6 +40,11 @@
  * from the clauses and comparing it with the partition bounds while also
  * taking into account strategies of the operators in the matched clauses.
  *
+ * make_partition_pruneinfo()
+ *
+ * Generates a PartitionPruneInfo node for use in the executor to allow it
+ * to perform partition pruning during execution.
+ *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
@@ -55,10 +60,14 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/executor.h"
+#include "executor/nodeSubplan.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -68,43 +77,6 @@
 #include "utils/lsyscache.h"
 
 /*
- * Stores clauses which were matched to a partition key.
- *
- * Each matching "operator" clause is stored in the 'keyclauses' list for the
- * partition key that it was matched to, except if the operator is <>, in
- * which case, the clause is added to the 'ne_clauses' list.
- *
- * Boolean OR clauses whose at least one argument clause matches a partition
- * key are added to the 'or_clauses' list.
- *
- * Based on a IS NULL or IS NOT NULL clause that was matched to a partition
- * key, the corresponding bit in 'keyisnull' or 'keyisnotnull' is set.  A bit
- * in 'keyisnotnull' may also be set when a strict OpExpr is encountered for
- * the given partition key.
- */
-typedef struct PartitionClauseInfo
-{
-	/* Lists of clauses indexed by the partition key */
-	List   *keyclauses[PARTITION_MAX_KEYS];
-
-	/* Each members is a List itself of a given OR clauses's arguments. */
-	List   *or_clauses;
-
-	/* List of clauses containing <> operator. */
-	List   *ne_clauses;
-
-	/* Nth (0 <= N < partnatts) bit set if the key is NULL or NOT NULL. */
-	Bitmapset   *keyisnull;
-	Bitmapset   *keyisnotnull;
-
-	/* True if at least one of above fields contains valid information. */
-	bool	foundkeyclauses;
-
-	/* True if mutually contradictory clauses were found. */
-	bool	constfalse;
-} PartitionClauseInfo;
-
-/*
  * Information about a clause matched with a partition key column kept to
  * avoid recomputing it in remove_redundant_clauses().
  */
@@ -140,14 +112,15 @@ static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
 								   List *or_args);
 static void remove_redundant_clauses(PartitionPruneContext *context,
 						 List **minimalclauses);
-static bool partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result);
+static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
-static bool partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value);
+static bool partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value);
 
 /*
  * prune_append_rel_partitions
@@ -192,6 +165,8 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.boundinfo = rel->boundinfo;
 		context.has_default_part = rel->has_default_part;
 		context.partition_qual = rel->partition_qual;
+		context.econtext = NULL;
+		context.paramids = NULL;
 
 		/* process clauses; context.clauseinfo will be set */
 		generate_partition_clauses(&context, clauses);
@@ -558,6 +533,10 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				pc->inputcollid = opclause->inputcollid;
 				pc->value = valueexpr;
 
+				if (IsA(valueexpr, Param))
+					context->paramids = bms_add_member(context->paramids,
+											((Param *) valueexpr)->paramid);
+
 				/*
 				 * We don't turn a <> operator clause into a key right away.
 				 * Instead, the caller will hand over such clauses to
@@ -719,6 +698,11 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 														 leftop, rightop,
 														 InvalidOid,
 														 saop_coll);
+
+					if (IsA(rightop, Param))
+						context->paramids = bms_add_member(context->paramids,
+												((Param *) rightop)->paramid);
+
 					elem_clauses = lappend(elem_clauses, elem_clause);
 				}
 
@@ -969,7 +953,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				if (hash_clause == NULL)
 					hash_clause = pc;
 				/* check if another clause would contradict the one we have */
-				else if (partition_cmp_args(context->parttypid[i],
+				else if (partition_cmp_args(context,
+											context->parttypid[i],
 											context->partopfamily[i],
 											pc, pc, hash_clause,
 											&test_result))
@@ -1026,7 +1011,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * then because 7 < 5 is false, we leave a < 5 where it is and
 				 * effectively discard a < 7 as being redundant.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   pc, pc, btree_clauses[s],
 									   &test_result))
@@ -1083,7 +1069,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * eq clause is a = 3, then because 3 < 5, we no longer need
 				 * a < 5, because a = 3 is more restrictive.
 				 */
-				if (partition_cmp_args(context->parttypid[i],
+				if (partition_cmp_args(context,
+									   context->parttypid[i],
 									   context->partopfamily[i],
 									   chk, eq, chk,
 									   &test_result))
@@ -1114,7 +1101,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *lt = btree_clauses[BTLessStrategyNumber - 1],
 					   *le = btree_clauses[BTLessEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   le, lt, le,
 								   &test_result))
@@ -1133,7 +1121,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *gt = btree_clauses[BTGreaterStrategyNumber - 1],
 					   *ge = btree_clauses[BTGreaterEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->parttypid[i],
+			if (partition_cmp_args(context,
+								   context->parttypid[i],
 								   context->partopfamily[i],
 								   ge, gt, ge,
 								   &test_result))
@@ -1171,9 +1160,9 @@ remove_redundant_clauses(PartitionPruneContext *context,
  * incompatible with the operator.
  */
 static bool
-partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result)
+partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result)
 {
 	Datum	left_value;
 	Datum	right_value;
@@ -1184,10 +1173,12 @@ partition_cmp_args(Oid parttypid, Oid partopfamily,
 	 * Try to extract an actual value from each arg.  This may fail if the
 	 * value is unknown in this context, in which case we cannot compare.
 	 */
-	if (!partkey_datum_from_expr(parttypid, leftarg->value, &left_value))
+	if (!partkey_datum_from_expr(context, parttypid, leftarg->value,
+		&left_value))
 		return false;
 
-	if (!partkey_datum_from_expr(parttypid, rightarg->value, &right_value))
+	if (!partkey_datum_from_expr(context, parttypid, rightarg->value,
+		&right_value))
 		return false;
 
 	/*
@@ -1310,12 +1301,16 @@ extract_bounding_datums(PartitionPruneContext *context,
 				case PART_OP_EQUAL:
 					Assert(incl);
 					if (need_next_eq &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1323,7 +1318,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 					}
 
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1333,7 +1330,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_LESS:
 					if (need_next_max &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1345,7 +1344,9 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_GREATER:
 					if (need_next_min &&
-						partkey_datum_from_expr(context->parttypid[i], value,
+						partkey_datum_from_expr(context,
+												context->parttypid[i],
+												value,
 												&keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1390,8 +1391,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 			PartClause *pc = (PartClause *) lfirst(lc);
 			Datum	datum;
 
-			if (partkey_datum_from_expr(context->parttypid[0], pc->value,
-										&datum))
+			if (partkey_datum_from_expr(context, context->parttypid[0],
+										pc->value, &datum))
 				keys->ne_datums[i++] = datum;
 		}
 		keys->n_ne_datums = i;
@@ -1471,7 +1472,8 @@ partition_op_strategy(char part_strategy, PartClause *pc, bool *incl)
  * set.  True is returned otherwise.
  */
 static bool
-partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -1495,25 +1497,193 @@ partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value)
 		if (expr == NULL)
 			return false;
 
-		/*
-		 * Transform into a form that the following code can do something
-		 * useful with.
-		 */
-		expr = evaluate_expr(expr,
-							 exprType((Node *) expr),
-							 exprTypmod((Node *) expr),
-							 exprCollation((Node *) expr));
+		if (context->planstate && context->econtext)
+		{
+			ExprState *exprstate;
+			bool isNull;
+
+			exprstate = ExecInitExpr(expr, context->planstate);
+
+			*value = ExecEvalExprSwitchContext(exprstate,
+											   context->econtext,
+											   &isNull);
+
+			if (isNull)
+				return false;
+
+			return true;
+		}
+		else
+		{
+			/*
+			 * Transform into a form that the following code can do something
+			 * useful with.
+			 */
+			expr = evaluate_expr(expr,
+								 exprType((Node *) expr),
+								 exprTypmod((Node *) expr),
+								 exprCollation((Node *) expr));
+		}
 	}
 
 	/*
 	 * Add more expression types here as needed to support the requirements
 	 * of the higher-level code.
 	 */
-	if (IsA(expr, Const))
+	switch (nodeTag(expr))
 	{
-		*value = ((Const *) expr)->constvalue;
-		return true;
+		case T_Const:
+			*value = ((Const *) expr)->constvalue;
+			return true;
+
+		case T_Param:
+			if (context->planstate && context->econtext)
+			{
+				ExprState *exprstate;
+				bool isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+				*value = ExecEvalExprSwitchContext(exprstate,
+												 context->econtext,
+												 &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+			}
+
+		default:
+			return false;
+	}
+}
+
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes.  This is required in order to allow
+ *		us to perform any further partition pruning during execution.
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	check_stack_depth();
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->relid = rel->relid;
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->allsubnodes = NULL;
+	pinfo->nparts = nparts;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/* Determine the element in part_rel which belongs to this subpath. */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
 	}
 
-	return false;
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			List *subprunequal;
+
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			subprunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																subprunequal);
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
+	}
+
+	return pinfo;
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fe3b458..448c05a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,36 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time.  Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths.  We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3596,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index f3063be..b3888d9 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1988,9 +1988,6 @@ set_baserel_partition_key_exprs(Relation relation,
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int			partnatts;
-	int			cnt;
-	List	  **partexprs;
-	ListCell   *lc;
 	Index		varno = rel->relid;
 
 	Assert(IS_SIMPLE_REL(rel) && rel->relid > 0);
@@ -1999,39 +1996,8 @@ set_baserel_partition_key_exprs(Relation relation,
 	Assert(partkey != NULL);
 
 	partnatts = partkey->partnatts;
-	partexprs = (List **) palloc(sizeof(List *) * partnatts);
-	lc = list_head(partkey->partexprs);
-
-	for (cnt = 0; cnt < partnatts; cnt++)
-	{
-		Expr	   *partexpr;
-		AttrNumber	attno = partkey->partattrs[cnt];
-
-		if (attno != InvalidAttrNumber)
-		{
-			/* Single column partition key is stored as a Var node. */
-			Assert(attno > 0);
-
-			partexpr = (Expr *) makeVar(varno, attno,
-										partkey->parttypid[cnt],
-										partkey->parttypmod[cnt],
-										partkey->parttypcoll[cnt], 0);
-		}
-		else
-		{
-			if (lc == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			/* Re-stamp the expression with given varno. */
-			partexpr = (Expr *) copyObject(lfirst(lc));
-			ChangeVarNodes((Node *) partexpr, 1, varno, 0);
-			lc = lnext(lc);
-		}
-
-		partexprs[cnt] = list_make1(partexpr);
-	}
 
-	rel->partexprs = partexprs;
+	rel->partexprs = build_partition_expressions(partkey, varno);
 
 	/*
 	 * A base relation can not have nullable partition key expressions. We
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index ed27ca9..2fc2f32 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -70,6 +71,12 @@ typedef struct PartitionPruneContext
 
 	/* Information about matched clauses */
 	struct PartitionClauseInfo *clauseinfo;
+
+	PlanState	*planstate;
+	ExprContext *econtext;
+
+	/* ParamIds of clauses being used to determine partitions */
+	Bitmapset *paramids;
 } PartitionPruneContext;
 
 /*
@@ -165,4 +172,6 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 extern Bitmapset *get_partitions_for_keys(PartitionPruneContext *context,
 						PartScanKeyInfo *keys);
 
+extern List **build_partition_expressions(PartitionKey partkey, Index varno);
+
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index a953820..7db3a79 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1010,11 +1010,14 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
 struct AppendState;
 typedef struct AppendState AppendState;
+struct PartitionPruneContextCache;
+typedef struct PartitionPruneContextCache PartitionPruneContextCache;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
 
@@ -1026,6 +1029,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	PartitionPruneContextCache *contextcache; /* cache of prune contexts */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c097da6..d693f37 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f2e19ea..2264d54 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -250,6 +250,13 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition index into subplan indexes.  This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..ffb6daf 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,27 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into an Append node's subplan index.  This structure is used to
+ * recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	int			relid;		/* relation index of parent partition rel */
+	Oid			parentoid;	/* Oid of parent partition rel */
+	List	   *prunequal;	/* qual list for pruning partitions */
+	Bitmapset  *allsubnodes;	/* All subnode indexes at this level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 5579940..4d17f9a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1294,6 +1294,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f549..bedffc4 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 2b84ed9..a761e65 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -16,10 +16,52 @@
 
 #include "catalog/partition.h"
 
+/*
+ * Stores clauses which were matched to a partition key.
+ *
+ * Each matching "operator" clause is stored in the 'keyclauses' list for the
+ * partition key that it was matched to, except if the operator is <>, in
+ * which case, the clause is added to the 'ne_clauses' list.
+ *
+ * Boolean OR clauses whose at least one argument clause matches a partition
+ * key are added to the 'or_clauses' list.
+ *
+ * Based on a IS NULL or IS NOT NULL clause that was matched to a partition
+ * key, the corresponding bit in 'keyisnull' or 'keyisnotnull' is set.  A bit
+ * in 'keyisnotnull' may also be set when a strict OpExpr is encountered for
+ * the given partition key.
+ */
+typedef struct PartitionClauseInfo
+{
+	/* Lists of clauses indexed by the partition key */
+	List   *keyclauses[PARTITION_MAX_KEYS];
+
+	/* Each members is a List itself of a given OR clauses's arguments. */
+	List   *or_clauses;
+
+	/* List of clauses containing <> operator. */
+	List   *ne_clauses;
+
+	/* Nth (0 <= N < partnatts) bit set if the key is NULL or NOT NULL. */
+	Bitmapset   *keyisnull;
+	Bitmapset   *keyisnotnull;
+
+	/* True if at least one of above fields contains valid information. */
+	bool	foundkeyclauses;
+
+	/* True if mutually contradictory clauses were found. */
+	bool	constfalse;
+} PartitionClauseInfo;
+
 extern Relids prune_append_rel_partitions(PlannerInfo *root,
 							RelOptInfo *rel);
 extern void generate_partition_clauses(PartitionPruneContext *context,
 							List *clauses);
 extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context);
 
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index ef7173f..bde1858 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 948cad4..424b3a7 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1475,3 +1475,1118 @@ explain (costs off) select * from like_op_noprune where a like '%BC';
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+deallocate q1;
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 08fc2db..73b4d10 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -253,3 +253,315 @@ create table like_op_noprune2 partition of like_op_noprune for values in ('BCD')
 explain (costs off) select * from like_op_noprune where a like '%BC';
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
-- 
1.9.5.msysgit.1

v12-0003-Pre-process-OR-clauses-and-store-in-the-Partitio.patchapplication/octet-stream; name=v12-0003-Pre-process-OR-clauses-and-store-in-the-Partitio.patchDownload
From 0e03546e6a0107e3d1371a7887f661bbb9484f53 Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Sat, 24 Feb 2018 00:21:40 +1300
Subject: [PATCH v12 3/3] Pre-process OR clauses and store in the
 PartitionClauseInfo

Previously OR clauses were not matched up to the partition keys during the
initial stages of partition elimination.  The work of processing these was
left until a later stage of the partition elimination function set.  This
might have been fine for performing partition elimination at planning time,
but when performing at execution time it meant that the OR clauses had to be
matched to the partition keys each time we had to redetermine the matching
partitions.  During execution this event occurs each time a parameter whose
value could affect the matching partitions changes, so in cases like
parameterized nested loops it could be every tuple.  Not processing the OR
clauses was also not of much use since we must know all parameter IDs matching
partition keys in advance, so that we're properly able to determine which
parameter changes we must redetermine the matching partitions again for.

Quite possibly this refactor should be tweaked and made part of the faster
partition pruning patch set.
---
 src/backend/executor/nodeAppend.c      |   8 +-
 src/backend/optimizer/util/partprune.c | 174 +++++++++++++++++++--------------
 src/include/optimizer/partprune.h      |  11 ++-
 3 files changed, 114 insertions(+), 79 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7171682..7658906 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -83,6 +83,7 @@ struct ParallelAppendState
 struct PartitionPruneContextCache
 {
 	PartitionPruneContext *context;
+	PartitionClauseInfo *partclauseinfo;
 	PartitionPruneContextCache *subcache;
 };
 
@@ -714,7 +715,7 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 		context->econtext = node->ps.ps_ExprContext;
 		context->paramids = NULL;
 
-		generate_partition_clauses(context, pinfo->prunequal);
+		ctxcache->partclauseinfo = generate_partition_clauses(context, pinfo->prunequal);
 
 		node->part_prune_params = bms_add_members(node->part_prune_params,
 												  context->paramids);
@@ -730,7 +731,7 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 	 * Detect if any impossibilities were discovered during
 	 * generate_partition_clauses
 	 */
-	if (context->clauseinfo->constfalse)
+	if (ctxcache->partclauseinfo->constfalse)
 	{
 		bms_free(*validsubplans);
 		*validsubplans = NULL;
@@ -747,7 +748,8 @@ set_valid_runtime_subplans_recurse(AppendState *node,
 	 * params.
 	 */
 	if (!bms_is_empty(context->paramids))
-		partset = get_partitions_from_clauses(context);
+		partset = get_partitions_from_clauses(context,
+											  ctxcache->partclauseinfo);
 	else
 		partset = pinfo->allsubnodes;
 
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 21e86ff..5b2ddad 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -24,21 +24,19 @@
  * pruning from a list of clauses containing clauses that reference a given
  * partitioned table.  For example, prune_append_rel_partitions() calls this
  * function, because a partitioned table's rel->baserestrictinfo may contain
- * clauses that might be useful for partitioning.  Caller must have set up a
- * valid partition pruning context in the form of struct PartitionPruneContext,
- * that is, each of its fields other other than clauseinfo must be valid before
- * calling here.  After extracting relevant clauses, clauseinfo is filled with
- * information that will be used for actual pruning.
+ * clauses that might be useful for partitioning.  The list of clauses is
+ * processed and a PartitionClauseInfo is returned which contains details of
+ * any clauses which could be matched to the partition keys of the relation
+ * defined in the context.
  *
  * get_partitions_from_clauses()
  *
- * This is to be called to prune partitions based on relevant partitioning
- * clauses.  Caller must have called generate_partition_clauses() at least
- * once and hence a valid partition pruning context must have already been
- * created.  Especially, PartitionPruneContext.clauseinfo must contain valid
- * information.  Partition pruning proceeds by extracting constant values
- * from the clauses and comparing it with the partition bounds while also
- * taking into account strategies of the operators in the matched clauses.
+ * This is to be called to prune partitions based on 'partclauseinfo'.  Caller
+ * must have called generate_partition_clauses() in order to have generated
+ * a valid PartitionClauseInfo.  Partition pruning proceeds by extracting
+ * constant values from the clauses and comparing it with the partition bounds
+ * while also taking into account strategies of the operators in the matched
+ * clauses.
  *
  * make_partition_pruneinfo()
  *
@@ -104,18 +102,22 @@ typedef enum PartOpStrategy
 	PART_OP_GREATER
 } PartOpStrategy;
 
-static void extract_partition_clauses(PartitionPruneContext *context,
-						   List *clauses);
+static PartitionClauseInfo *extract_partition_clauses(
+						  PartitionPruneContext *context,
+						  List *clauses);
 static bool match_boolean_partition_clause(Expr *clause, Expr *partkey,
 							   Expr **rightop);
 static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
-								   List *or_args);
+							List *or_args,
+							List *or_partclauselist);
 static void remove_redundant_clauses(PartitionPruneContext *context,
+						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses);
 static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
 				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
 				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
+						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
@@ -144,6 +146,7 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 	else
 	{
 		PartitionPruneContext context;
+		PartitionClauseInfo *partclauseinfo;
 		int		partnatts = rel->part_scheme->partnatts,
 				i;
 
@@ -168,13 +171,16 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.econtext = NULL;
 		context.paramids = NULL;
 
-		/* process clauses; context.clauseinfo will be set */
-		generate_partition_clauses(&context, clauses);
+		/* process clauses */
+		partclauseinfo = generate_partition_clauses(&context, clauses);
 
-		if (!context.clauseinfo->constfalse)
+		if (!partclauseinfo->constfalse)
 		{
 			/* Actual pruning happens here. */
-			Bitmapset *partindexes = get_partitions_from_clauses(&context);
+			Bitmapset *partindexes;
+
+			partindexes = get_partitions_from_clauses(&context,
+													  partclauseinfo);
 
 			/* Add selected partitions' RT indexes to result. */
 			i = -1;
@@ -188,13 +194,10 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 
 /*
  * generate_partition_clauses
- *		Analyzes clauses to find those that match the partition key and sets
- *		context->clauseinfo
- *
- * Ideally, this should be called only once for a given query and a given
- * partitioned table.
+ *		Processes 'clauses' and returns a PartitionClauseInfo which contains
+ *		the details of any clauses which were matched to partition keys.
  */
-void
+PartitionClauseInfo *
 generate_partition_clauses(PartitionPruneContext *context, List *clauses)
 {
 	/* The clauses list may be modified below, so better make a copy. */
@@ -227,25 +230,26 @@ generate_partition_clauses(PartitionPruneContext *context, List *clauses)
 		clauses = list_concat(clauses, partqual);
 	}
 
-	/* And away we go to do the real work; context->clauseinfo will be set */
-	extract_partition_clauses(context, clauses);
+	/* pre-process the clauses and generate the PartitionClauseInfo */
+	return extract_partition_clauses(context, clauses);
 }
 
 /*
  * get_partitions_from_clauses
  *		Determine partitions that could possible contain a record that
- *		satisfies clauses as described in context->clauseinfo
+ *		satisfies clauses as described in partclauseinfo
  *
  * Returns a Bitmapset of the matching partition indexes, or NULL if none can
  * match.
  */
 Bitmapset *
-get_partitions_from_clauses(PartitionPruneContext *context)
+get_partitions_from_clauses(PartitionPruneContext *context,
+							PartitionClauseInfo *partclauseinfo)
 {
-	PartitionClauseInfo	*partclauseinfo = context->clauseinfo;
 	PartScanKeyInfo		keys;
 	Bitmapset 		   *result;
-	ListCell *lc;
+	ListCell		   *lc;
+	ListCell		   *lc2;
 
 	Assert(partclauseinfo != NULL);
 	Assert(!partclauseinfo->constfalse);
@@ -260,17 +264,18 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 		List *minimalclauses[PARTITION_MAX_KEYS];
 
 		/*
-		 * For each partition key column, populate its slot in minimalclauses
-		 * with the most restrictive of the clauses from the corresponding
-		 * list in context->clauseinfo.
+		 * For each partition key column, populate its element in
+		 * minimalclauses with the most restrictive set of the clauses from
+		 * the corresponding partition key in partclauseinfo.
 		 */
-		remove_redundant_clauses(context, minimalclauses);
+		remove_redundant_clauses(context, partclauseinfo, minimalclauses);
 
 		/* Did remove_redundant_clauses find any contradicting clauses? */
 		if (partclauseinfo->constfalse)
 			return NULL;
 
-		if (extract_bounding_datums(context, minimalclauses, &keys))
+		if (extract_bounding_datums(context, partclauseinfo, minimalclauses,
+			&keys))
 		{
 			result = get_partitions_for_keys(context, &keys);
 
@@ -292,12 +297,14 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 	}
 
 	/* Now apply the OR clauses. */
-	foreach(lc, partclauseinfo->or_clauses)
+	forboth(lc, partclauseinfo->or_clauses, lc2,
+			partclauseinfo->or_partclauseinfos)
 	{
 		List *or_args = (List *) lfirst(lc);
+		List *or_partclauselist = lfirst(lc2);
 		Bitmapset *or_parts;
 
-		or_parts = get_partitions_from_or_args(context, or_args);
+		or_parts = get_partitions_from_or_args(context, or_args, or_partclauselist);
 
 		/*
 		 * Clauses in or_clauses are mutually conjunctive and also in
@@ -327,26 +334,27 @@ get_partitions_from_clauses(PartitionPruneContext *context)
 /*
  * extract_partition_clauses
  *		Processes 'clauses' to extract clause matching the partition key.
- *		This adds matched clauses to the list corresponding to particular key
- *		in context->clauseinfo.  Also collects other useful clauses to assist
- *		in partition elimination, such as OR clauses, clauses containing <>
- *		operator, and IS [NOT] NULL clauses
+ *		Returns a PartitionClauseInfo which stores the clauses which were
+ *		matched to the partition key.  The PartitionClauseInfo also collects
+ *		other useful clauses to assist in partition elimination, such as OR
+ *		clauses, clauses containing <> operator, and IS [NOT] NULL clauses
  *
  * We may also discover some contradiction in the clauses which means that no
- * partition can possibly match.  In this case, the function sets
- * context->clauseinfo's 'constfalse' to true and exits immediately without
- * processing any further clauses.  In this case, the caller must be careful
- * not to assume the context->clauseinfo is fully populated with all clauses.
+ * partition can possibly match.  In this case, the function sets the
+ * returned PartitionClauseInfo's 'constfalse' to true and exits immediately
+ * without processing any further clauses.  In this case, the caller must be
+ * careful not to assume the return value is fully populated with all clauses.
  */
-static void
+static PartitionClauseInfo *
 extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 {
 	PartitionClauseInfo *partclauseinfo;
 	ListCell   *lc;
 
-	context->clauseinfo = partclauseinfo = palloc(sizeof(PartitionClauseInfo));
+	partclauseinfo = palloc(sizeof(PartitionClauseInfo));
 	memset(partclauseinfo->keyclauses, 0, sizeof(partclauseinfo->keyclauses));
 	partclauseinfo->or_clauses = NIL;
+	partclauseinfo->or_partclauseinfos = NIL;
 	partclauseinfo->ne_clauses = NIL;
 	partclauseinfo->keyisnull = NULL;
 	partclauseinfo->keyisnotnull = NULL;
@@ -367,7 +375,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				!DatumGetBool(((Const *) clause)->constvalue))
 			{
 				partclauseinfo->constfalse = true;
-				return;
+				return partclauseinfo;
 			}
 		}
 
@@ -558,7 +566,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				if (bms_is_member(i, partclauseinfo->keyisnull))
 				{
 					partclauseinfo->constfalse = true;
-					return;
+					return partclauseinfo;
 				}
 				/* Record that a strict clause has been seen for this key */
 				partclauseinfo->keyisnotnull =
@@ -734,7 +742,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 						if (bms_is_member(i, partclauseinfo->keyisnotnull))
 						{
 							partclauseinfo->constfalse = true;
-							return;
+							return partclauseinfo;
 						}
 						partclauseinfo->keyisnull =
 									bms_add_member(partclauseinfo->keyisnull,
@@ -746,7 +754,7 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 						if (bms_is_member(i, partclauseinfo->keyisnull))
 						{
 							partclauseinfo->constfalse = true;
-							return;
+							return partclauseinfo;
 						}
 
 						partclauseinfo->keyisnotnull =
@@ -760,6 +768,31 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 			partclauseinfo->foundkeyclauses = true;
 		}
 	}
+
+	/*
+	 * Now pre-process any OR clauses found above and generate
+	 * PartitionClauseInfos for them.
+	 */
+	foreach(lc, partclauseinfo->or_clauses)
+	{
+		List *or_args = lfirst(lc);
+		List *pclauselist = NIL;
+		ListCell *lc2;
+
+		foreach (lc2, or_args)
+		{
+			List *clauses = list_make1(lfirst(lc2));
+			PartitionClauseInfo *orpartclauseinfo;
+
+			orpartclauseinfo = extract_partition_clauses(context, clauses);
+			pclauselist = lappend(pclauselist, orpartclauseinfo);
+		}
+
+		partclauseinfo->or_partclauseinfos =
+					lappend(partclauseinfo->or_partclauseinfos, pclauselist);
+	}
+
+	return partclauseinfo;
 }
 
 /*
@@ -829,10 +862,11 @@ match_boolean_partition_clause(Expr *clause, Expr *partkey,
  * clause in or_args.
  */
 static Bitmapset *
-get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
+get_partitions_from_or_args(PartitionPruneContext *context, List *or_args,
+							List *or_partclauselist)
 {
 	Bitmapset	   *result = NULL;
-	ListCell	   *lc;
+	ListCell	   *lc, *lc2;
 
 	/*
 	 * When matching an OR expression, it is only checked if at least one of
@@ -843,20 +877,13 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 	 * clause refutes its partition constraint, that is, we can eliminate all
 	 * of its partitions.
 	 */
-	foreach(lc, or_args)
+	forboth(lc, or_args, lc2, or_partclauselist)
 	{
 		List *clauses = list_make1(lfirst(lc));
-		PartitionPruneContext subcontext;
+		PartitionClauseInfo *or_pclause = lfirst(lc2);
 		Bitmapset *arg_partset;
 
-		/*
-		 * All fields except clauseinfo are same as in the parent context,
-		 * which will be set by calling extract_partition_clauses().
-		 */
-		memcpy(&subcontext, context, sizeof(PartitionPruneContext));
-		extract_partition_clauses(&subcontext, clauses);
-
-		if (!subcontext.clauseinfo->foundkeyclauses)
+		if (!or_pclause->foundkeyclauses)
 		{
 			List *partconstr = context->partition_qual;
 
@@ -873,8 +900,8 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 			return bms_add_range(NULL, 0, context->nparts - 1);
 		}
 
-		if (!subcontext.clauseinfo->constfalse)
-			arg_partset = get_partitions_from_clauses(&subcontext);
+		if (!or_pclause->constfalse)
+			arg_partset = get_partitions_from_clauses(context, or_pclause);
 		else
 			arg_partset = NULL;
 
@@ -887,8 +914,8 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
 
 /*
  * remove_redundant_clauses
- *		Processes the clauses contained in context->clauseinfo to remove the
- *		ones that are superseeded by other clauses which are more restrictive.
+ *		Process 'partpruneinfo' to remove the clauses that are superseeded
+ *		by other clauses which are more restrictive.
  *
  * Finished lists of clauses are returned in *minimalclauses which is an array
  * with one slot for each of the partition keys.
@@ -900,16 +927,16 @@ get_partitions_from_or_args(PartitionPruneContext *context, List *or_args)
  * that the clauses cannot possibly match any partition.  Impossible clauses
  * include things like: x = 1 AND x = 2, x > 0 and x < 10.  The function
  * returns right after finding such a clause and before returning, sets
- * constfalse in context->clauseinfo to inform the caller that we found such
+ * constfalse in 'partclauseinfo' to inform the caller that we found such
  * clause.
  */
 static void
 remove_redundant_clauses(PartitionPruneContext *context,
+						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses)
 {
 	PartClause *hash_clause,
 			   *btree_clauses[BTMaxStrategyNumber];
-	PartitionClauseInfo *partclauseinfo = context->clauseinfo;
 	ListCell *lc;
 	int		s;
 	int		i;
@@ -1220,8 +1247,9 @@ partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
 
 /*
  * extract_bounding_datums
- *		Process clauses in context->clauseinfo and populate 'keys' with all
- *		min/max/equal/not-equal values that we're able to determine.
+ *		Process 'clauseinfo' and populate 'keys' with all
+ *		min/max/equal/not-equal values that we're able to
+ *		determine.
  *
  * *minimalclauses is an array with partnatts members, each of which is a list
  * of the most restrictive clauses of each operator strategy for the given
@@ -1237,9 +1265,9 @@ partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
  */
 static bool
 extract_bounding_datums(PartitionPruneContext *context,
+						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys)
 {
-	PartitionClauseInfo *clauseinfo = context->clauseinfo;
 	bool		need_next_eq,
 				need_next_min,
 				need_next_max;
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index a761e65..ff68d4a 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -39,6 +39,9 @@ typedef struct PartitionClauseInfo
 	/* Each members is a List itself of a given OR clauses's arguments. */
 	List   *or_clauses;
 
+	/* each OR clause processed into a PartitionClauseInfo */
+	List *or_partclauseinfos;
+
 	/* List of clauses containing <> operator. */
 	List   *ne_clauses;
 
@@ -55,9 +58,11 @@ typedef struct PartitionClauseInfo
 
 extern Relids prune_append_rel_partitions(PlannerInfo *root,
 							RelOptInfo *rel);
-extern void generate_partition_clauses(PartitionPruneContext *context,
-							List *clauses);
-extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context);
+extern PartitionClauseInfo *generate_partition_clauses(
+						   PartitionPruneContext *context,
+						   List *clauses);
+extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context,
+							PartitionClauseInfo *partclauseinfo);
 
 extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
 						 RelOptInfo *rel,
-- 
1.9.5.msysgit.1

#85David Rowley
david.rowley@2ndquadrant.com
In reply to: Jesper Pedersen (#81)
Re: [HACKERS] Runtime Partition Pruning

On 23 February 2018 at 04:11, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Are UPDATE and DELETE suppose to be supported ?

To be honest, I had not even considered those. Without looking in
detail I imagine it may be possible to allow this simply by setting
the AppendPath->trypartitionprune in the correct cases in the
inheritence_planner(). I would need to look into this in some detail
to find out for sure.

Another case which likely is simple to implement is the exact same
processing for MergeAppends. I currently see no reason why the same
pruning cannot be done for subnodes of that node type too. I've just
not done so yet. I'd rather get more sanity check reviews on the
current scope of the patch before I widen it out to other areas, but
at the same time also don't want to leave very simple things to PG12
which can easily be done in PG11. So I'll try to look at this and get
back to you, or perhaps release a new set of patches to support the
additional features.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#86Simon Riggs
simon@2ndquadrant.com
In reply to: David Rowley (#85)
Re: [HACKERS] Runtime Partition Pruning

On 23 February 2018 at 11:40, David Rowley <david.rowley@2ndquadrant.com> wrote:

On 23 February 2018 at 04:11, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Are UPDATE and DELETE suppose to be supported ?

To be honest, I had not even considered those.

I can say that I had considered those. Handling of UPDATE and DELETE
with partitions is significantly different, so its not just an
oversight its a different branch of the project.

We need SELECT to work first and then we have a chance of making
UPDATE/DELETE work.

--
Simon Riggs http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#87Amit Langote
amitlangote09@gmail.com
In reply to: Simon Riggs (#86)
Re: [HACKERS] Runtime Partition Pruning

(Sorry, I had mistakenly replied only to Simon on Friday)

On Fri, Feb 23, 2018 at 9:04 PM, Simon Riggs <simon@2ndquadrant.com> wrote:

On 23 February 2018 at 11:40, David Rowley <david.rowley@2ndquadrant.com> wrote:

On 23 February 2018 at 04:11, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Are UPDATE and DELETE suppose to be supported ?

To be honest, I had not even considered those.

I can say that I had considered those. Handling of UPDATE and DELETE
with partitions is significantly different, so its not just an
oversight its a different branch of the project.

We need SELECT to work first and then we have a chance of making
UPDATE/DELETE work.

+1

Thanks,
Amit

#88Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#85)
Re: [HACKERS] Runtime Partition Pruning

On 2018/02/23 20:40, David Rowley wrote:

On 23 February 2018 at 04:11, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Are UPDATE and DELETE suppose to be supported ?

To be honest, I had not even considered those. Without looking in
detail I imagine it may be possible to allow this simply by setting
the AppendPath->trypartitionprune in the correct cases in the
inheritence_planner(). I would need to look into this in some detail
to find out for sure.

I guess you meant in ModifyTablePath.

Thanks,
Amit

#89Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#84)
2 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/02/23 20:34, David Rowley wrote:

On 22 February 2018 at 22:31, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

Some comments:

* I noticed that the patch adds a function to bitmapset.c which you might
want to extract into its own patch, like your patch to add bms_add_range()
that got committed as 84940644d [2].

I've made that 0001 in the series

Thanks.

I've attached an updated set of patches.

I hope this also addresses Rajkumar reported crash. I ended up making
some changes to how the Param values are determined by reusing more of
the existing executor code rather than duplicating it in
partkey_datum_from_expr. I really could use a sanity check on my
changes to that function now, especially the cross type portion.

I've incorporated portions of 0002 and 0003 into my patch on the other
thread (v34) posted at [1]/messages/by-id/158f04ce-9deb-0457-ddcc-78fb73db4ebc@lab.ntt.co.jp. That is, mostly the changes around handling
OR clauses and interface changes resulting from it.

Attached are revised version of your patches after the aforementioned
rearrangements. Note that after I took out the optimizer portion of the
0003 patch to incorporate it into my patch (OR clause processing bits),
not much was left in it, so I squashed it into 0002. So there are only
0001 and 0002.

As a review comment on 0002, I think trypartitionprune is better written
as try_partition_prune.

Thanks,
Amit

[1]: /messages/by-id/158f04ce-9deb-0457-ddcc-78fb73db4ebc@lab.ntt.co.jp
/messages/by-id/158f04ce-9deb-0457-ddcc-78fb73db4ebc@lab.ntt.co.jp

Attachments:

v13-0001-Add-bms_prev_member-function.patchtext/plain; charset=UTF-8; name=v13-0001-Add-bms_prev_member-function.patchDownload
From ad0083cfecfb5a27a1e912bc7c4244340a674a46 Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Fri, 23 Feb 2018 23:49:05 +1300
Subject: [PATCH v13 1/2] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..3578c8fffc 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.11.0

v13-0002-Allow-partition-elimination-to-occur-during-exec.patchtext/plain; charset=UTF-8; name=v13-0002-Allow-partition-elimination-to-occur-during-exec.patchDownload
From 4bc98cfd662d2d8022fabd8cd2c515ce22e9e8ff Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Tue, 27 Feb 2018 15:20:02 +0900
Subject: [PATCH v13 2/2] Allow partition elimination to occur during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using information which is available during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to participate too.  We can also perform partition elimination in
the following cases:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Unlike the case of partition elimination during query planning, when the
elimination takes place during execution, the Append's subnode are still visible
in the EXPLAIN output.  In order to determine if pruning has actually taken
place the EXPLAIN ANALYZE must be viewed.  If a certain Append subplan was
never executed due to elimination of the partition then the execution timing
area will state "(never executed)".  Whereas, if, for example in the case of
parameterized nested loops, the number of loops stated in the EXPLAIN ANALYZE
output for certain subplans may appear lower than others due to the subplan
having been scanned fewer times.

For now, only the Append nodes when used in a SELECT query allows this
execution time elimination to take place.  The features added here are likely
easily extendable into MergeAppend and possibly also for DML statements too,
such as UPDATE and DELETE.
---
 src/backend/catalog/partition.c               |   50 ++
 src/backend/executor/nodeAppend.c             |  330 +++++++-
 src/backend/nodes/copyfuncs.c                 |   32 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   49 +-
 src/backend/optimizer/plan/planner.c          |    3 +-
 src/backend/optimizer/prep/prepunion.c        |    4 +-
 src/backend/optimizer/util/clauses.c          |   19 +
 src/backend/optimizer/util/partprune.c        |  266 +++++-
 src/backend/optimizer/util/pathnode.c         |   37 +-
 src/backend/optimizer/util/plancat.c          |   36 +-
 src/include/catalog/partition.h               |    9 +
 src/include/nodes/execnodes.h                 |    8 +
 src/include/nodes/nodes.h                     |    1 +
 src/include/nodes/plannodes.h                 |    7 +
 src/include/nodes/primnodes.h                 |   23 +
 src/include/nodes/relation.h                  |    2 +
 src/include/optimizer/clauses.h               |    3 +
 src/include/optimizer/partprune.h             |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1115 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  312 +++++++
 23 files changed, 2219 insertions(+), 108 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 6a2761c350..c4542b00cf 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -4203,3 +4203,53 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * build_partition_expressions
+ *		Build a list of partition key expressions.  Plain attibute keys will
+ *		be build as Vars with the varno set to 'varno'.
+ */
+List **
+build_partition_expressions(PartitionKey partkey, Index varno)
+{
+	int			partnatts;
+	int			cnt;
+	List	  **partexprs;
+	ListCell   *lc;
+
+	partnatts = partkey->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * partnatts);
+	lc = list_head(partkey->partexprs);
+
+	for (cnt = 0; cnt < partnatts; cnt++)
+	{
+		Expr	   *partexpr;
+		AttrNumber	attno = partkey->partattrs[cnt];
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Assert(attno > 0);
+
+			partexpr = (Expr *) makeVar(varno, attno,
+										partkey->parttypid[cnt],
+										partkey->parttypmod[cnt],
+										partkey->parttypcoll[cnt], 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expression with given varno. */
+			partexpr = (Expr *) copyObject(lfirst(lc));
+			if (varno != 1)
+				ChangeVarNodes((Node *) partexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt] = list_make1(partexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 7a3dd2ee2d..b1f147d357 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,6 +60,10 @@
 #include "executor/execdebug.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "optimizer/partprune.h"
+#include "optimizer/plancat.h"
+#include "utils/memutils.h"
 
 /* Shared state for parallel-aware Append. */
 struct ParallelAppendState
@@ -76,12 +80,26 @@ struct ParallelAppendState
 	bool		pa_finished[FLEXIBLE_ARRAY_MEMBER];
 };
 
+struct PartitionPruneContextCache
+{
+	PartitionPruneContext *context;
+	PartitionClauseInfo *partclauseinfo;
+	PartitionPruneContextCache *subcache;
+};
+
 #define INVALID_SUBPLAN_INDEX		-1
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void set_valid_runtime_subplans(AppendState *node);
+static void set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans);
+static void mark_invalid_subplans_as_finished(AppendState *node);
+
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -127,6 +145,34 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.ExecProcNode = ExecAppend;
 	appendstate->appendplans = appendplanstates;
 	appendstate->as_nplans = nplans;
+	appendstate->as_valid_subplans = NULL;
+	appendstate->part_prune_params = NULL; /* determined later */
+	appendstate->part_prune_info = node->part_prune_info;
+	appendstate->contextcache = NULL; /* populate this as needed below */
+
+	if (node->part_prune_info)
+	{
+		/*
+		 * When run-time partition pruning is enabled we make calls to a query
+		 * planner function to determine which partitions will match.  The
+		 * planner is not too careful about freeing memory, so we'll ensure we
+		 * call the function in a temporary memory context to avoid any memory
+		 * leaking in the executor's memory context.
+		 */
+		appendstate->prune_context =
+			AllocSetContextCreate(CurrentMemoryContext,
+								  "Partition Prune",
+								  ALLOCSET_DEFAULT_SIZES);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->prune_context = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
@@ -149,18 +195,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node
 	 */
+	ExecAssignExprContext(estate, &appendstate->ps);
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If parallel-aware, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +293,17 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they can reselected
+	 * for the new parameter values.
+	 */
+	if (bms_overlap(node->ps.chgParam, node->part_prune_params))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +323,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,22 +413,35 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	/* We should never see INVALID_SUBPLAN_INDEX in this case. */
-	Assert(whichplan >= 0 && whichplan <= node->as_nplans);
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
+	{
+		if (node->as_valid_subplans == NULL)
+			set_valid_runtime_subplans(node);
+
+		whichplan = -1;
+	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		if (whichplan >= node->as_nplans - 1)
-			return false;
-		node->as_whichplan++;
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -408,6 +474,17 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			set_valid_runtime_subplans(node);
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -460,6 +537,17 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		set_valid_runtime_subplans(node);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -525,3 +613,199 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * set_valid_runtime_subplans
+ *		Determine which subset of subplan nodes we need to scan based on
+ *		the details stored in node's 'part_prune_info'.  All subplans which
+ *		provably cannot possibly have matching records are eliminated and the
+ *		remainder are set in the AppendState's 'as_valid_subplans' variable.
+ */
+static void
+set_valid_runtime_subplans(AppendState *node)
+{
+	MemoryContext oldcontext;
+	Bitmapset *validsubplans = NULL;
+
+	/* Should never be called when already set */
+	Assert(node->as_valid_subplans == NULL);
+
+	if (!node->contextcache)
+		node->contextcache = palloc0(sizeof(PartitionPruneContextCache));
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(node->prune_context);
+
+	set_valid_runtime_subplans_recurse(node, node->part_prune_info,
+									   node->contextcache,
+									   &validsubplans);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	node->as_valid_subplans = bms_copy(validsubplans);
+
+	MemoryContextReset(node->prune_context);
+}
+
+static void
+set_valid_runtime_subplans_recurse(AppendState *node,
+								   PartitionPruneInfo *pinfo,
+								   PartitionPruneContextCache *ctxcache,
+								   Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context;
+	Bitmapset	   *partset;
+	int				i;
+
+	check_stack_depth();
+
+	/*
+	 * If the PartitionPruneContext has not yet been initialized for this rel
+	 * yet, then do that now.
+	 */
+	if (!ctxcache->context)
+	{
+		PartitionDesc	partdesc;
+		Relation		rel;
+		PartitionKey	partkey;
+		MemoryContext	oldContext;
+		List		  **partexprs;
+		int				partnatts;
+		int				i;
+
+		oldContext = MemoryContextSwitchTo(node->ps.state->es_query_cxt);
+
+		ctxcache->context = context = palloc(sizeof(PartitionPruneContext));
+		ctxcache->subcache = palloc0(sizeof(PartitionPruneContextCache) *
+									 pinfo->nparts);
+
+		rel = relation_open(pinfo->parentoid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->relid = pinfo->relid;
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+		partexprs = build_partition_expressions(partkey, pinfo->relid);
+		context->partkeys = (Expr **) palloc(sizeof(Expr *) * partnatts);
+
+		for (i = 0; i < partnatts; i++)
+			context->partkeys[i] = (Expr *) linitial(partexprs[i]);
+
+		context->partopfamily = partkey->partopfamily;
+		context->partopcintype = partkey->partopcintype;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		if (OidIsValid(get_default_oid_from_partdesc(partdesc)))
+			context->has_default_part = true;
+		else
+			context->has_default_part = false;
+
+		context->partition_qual = RelationGetPartitionQual(rel);
+
+		context->planstate = &node->ps;
+		context->econtext = node->ps.ps_ExprContext;
+		context->paramids = NULL;
+
+		ctxcache->partclauseinfo = generate_partition_clauses(context, pinfo->prunequal);
+
+		node->part_prune_params = bms_add_members(node->part_prune_params,
+												  context->paramids);
+
+		relation_close(rel, NoLock);
+
+		MemoryContextSwitchTo(oldContext);
+	}
+	else
+		context = ctxcache->context;
+
+	/*
+	 * Detect if any impossibilities were discovered during
+	 * generate_partition_clauses
+	 */
+	if (ctxcache->partclauseinfo->constfalse)
+	{
+		bms_free(*validsubplans);
+		*validsubplans = NULL;
+		return;
+	}
+
+	/*
+	 * 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 subpartitions as we may find their partitions keys match some
+	 * params.
+	 */
+	if (!bms_is_empty(context->paramids))
+		partset = get_partitions_from_clauses(context,
+											  ctxcache->partclauseinfo);
+	else
+		partset = pinfo->allsubnodes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pinfo->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pinfo->subnodeindex[i]);
+		else if (pinfo->subpartindex[i] != NULL)
+			set_valid_runtime_subplans_recurse(node,
+												pinfo->subpartindex[i],
+												&ctxcache->subcache[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing an Append subnode.
+			 * This shouldn't happen and could only happen if a more
+			 * restrictive clause list was used for partition elimination
+			 * during planning than what was used here.
+			 */
+			elog(ERROR, "partition missing from Append subplans");
+		}
+	}
+
+}
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->part_prune_info != NULL);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 169c697c08..629cdc05a0 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -244,6 +244,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_info);
 
 	return newnode;
 }
@@ -2132,6 +2133,34 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+	int i;
+
+	COPY_SCALAR_FIELD(relid);
+	COPY_SCALAR_FIELD(parentoid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_BITMAPSET_FIELD(allsubnodes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts *
+					   sizeof(PartitionPruneInfo *));
+
+	/*
+	 * The above copied the entire array, but we still need to create copies
+	 * of each PartitionPruneInfo contained in that array.
+	 */
+	for (i = 0; i < from->nparts; i++)
+	{
+		if (newnode->subpartindex[i] != NULL)
+			COPY_NODE_FIELD(subpartindex[i]);
+	}
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5028,6 +5057,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 08570ce25d..5bcb19947c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1587,7 +1587,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1629,8 +1629,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1678,7 +1678,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1734,7 +1734,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2000,7 +2000,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9ae1bf31d5..4adf6d07c2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -204,7 +205,8 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1022,6 +1024,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	PartitionPruneInfo *pinfo = NULL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1059,6 +1063,41 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+
+	if (best_path->trypartitionprune)
+	{
+		List	   *prunequal;
+
+		/* Not for join rels */
+		Assert(bms_membership(rel->relids) == BMS_SINGLETON);
+
+		prunequal =
+			extract_actual_clauses(best_path->path.parent->baserestrictinfo,
+								   false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist that could possibly be useful to use for
+		 * performing further partition pruning during execution, then
+		 * we'll generate a PartitionPruneInfo to store these quals and
+		 * allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			pinfo = make_partition_pruneinfo(root, best_path->path.parent,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
+	}
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1067,7 +1106,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   pinfo);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5315,7 +5355,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			PartitionPruneInfo *partpruneinfo)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5327,7 +5368,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_info = partpruneinfo;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1671f450b0..6d0d96cd81 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3737,7 +3737,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f01119eff1..146e202bb0 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -593,7 +593,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -705,7 +705,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 0c1f23951a..dccbcc19cc 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -2473,6 +2473,25 @@ eval_const_expressions(PlannerInfo *root, Node *node)
 }
 
 /*--------------------
+ * eval_const_expressions_from_list
+ *
+ * This is similar to eval_const_expression except that it takes ParamListInfo
+ * argument instead of PlannerInfo to create the context.
+ */
+Node *
+eval_const_expressions_from_list(ParamListInfo prmlist, Node *node)
+{
+	eval_const_expressions_context context;
+
+	context.boundParams = prmlist;	/* bound Params */
+	context.root = NULL;
+	context.active_fns = NIL;	/* nothing being recursively simplified */
+	context.case_val = NULL;	/* no CASE being examined */
+	context.estimate = false;	/* safe transformations only */
+	return eval_const_expressions_mutator(node, &context);
+}
+
+/*--------------------
  * estimate_expression_value
  *
  * This function attempts to estimate the value of an expression for
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 905bd3571c..a59bac005c 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -38,6 +38,11 @@
  * while also taking into account strategies of the operators in the matched
  * clauses.
  *
+ * make_partition_pruneinfo()
+ *
+ * Generates a PartitionPruneInfo node for use in the executor to allow it
+ * to perform partition pruning during execution.
+ *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
@@ -53,10 +58,14 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/executor.h"
+#include "executor/nodeSubplan.h"
+#include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -104,15 +113,16 @@ static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
 static void remove_redundant_clauses(PartitionPruneContext *context,
 						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses);
-static bool partition_cmp_args(Oid parttypid, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result);
+static bool partition_cmp_args(PartitionPruneContext *context, Oid parttypid,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
 						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
-static bool partkey_datum_from_expr(Oid parttypid, Expr *expr, Datum *value);
+static bool partkey_datum_from_expr(PartitionPruneContext *context, Oid parttypid,
+						Expr *expr, Datum *value);
 
 /*
  * prune_append_rel_partitions
@@ -158,6 +168,8 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.boundinfo = rel->boundinfo;
 		context.has_default_part = rel->has_default_part;
 		context.partition_qual = rel->partition_qual;
+		context.econtext = NULL;
+		context.paramids = NULL;
 
 		/* process clauses */
 		partclauseinfo = generate_partition_clauses(&context, clauses);
@@ -534,6 +546,10 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				pc->inputcollid = opclause->inputcollid;
 				pc->value = valueexpr;
 
+				if (IsA(valueexpr, Param))
+					context->paramids = bms_add_member(context->paramids,
+											((Param *) valueexpr)->paramid);
+
 				/*
 				 * We don't turn a <> operator clause into a key right away.
 				 * Instead, the caller will hand over such clauses to
@@ -695,6 +711,11 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 														 leftop, rightop,
 														 InvalidOid,
 														 saop_coll);
+
+					if (IsA(rightop, Param))
+						context->paramids = bms_add_member(context->paramids,
+												((Param *) rightop)->paramid);
+
 					elem_clauses = lappend(elem_clauses, elem_clause);
 				}
 
@@ -966,7 +987,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				if (hash_clause == NULL)
 					hash_clause = pc;
 				/* check if another clause would contradict the one we have */
-				else if (partition_cmp_args(context->partopcintype[i],
+				else if (partition_cmp_args(context,
+											context->partopcintype[i],
 											context->partopfamily[i],
 											pc, pc, hash_clause,
 											&test_result))
@@ -1023,7 +1045,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * then because 7 < 5 is false, we leave a < 5 where it is and
 				 * effectively discard a < 7 as being redundant.
 				 */
-				if (partition_cmp_args(context->partopcintype[i],
+				if (partition_cmp_args(context,
+									   context->partopcintype[i],
 									   context->partopfamily[i],
 									   pc, pc, btree_clauses[s],
 									   &test_result))
@@ -1080,7 +1103,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * eq clause is a = 3, then because 3 < 5, we no longer need
 				 * a < 5, because a = 3 is more restrictive.
 				 */
-				if (partition_cmp_args(context->partopcintype[i],
+				if (partition_cmp_args(context,
+									   context->partopcintype[i],
 									   context->partopfamily[i],
 									   chk, eq, chk,
 									   &test_result))
@@ -1111,7 +1135,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *lt = btree_clauses[BTLessStrategyNumber - 1],
 					   *le = btree_clauses[BTLessEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->partopcintype[i],
+			if (partition_cmp_args(context,
+								   context->partopcintype[i],
 								   context->partopfamily[i],
 								   le, lt, le,
 								   &test_result))
@@ -1130,7 +1155,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *gt = btree_clauses[BTGreaterStrategyNumber - 1],
 					   *ge = btree_clauses[BTGreaterEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->partopcintype[i],
+			if (partition_cmp_args(context,
+								   context->partopcintype[i],
 								   context->partopfamily[i],
 								   ge, gt, ge,
 								   &test_result))
@@ -1168,7 +1194,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
  * incompatible with the operator.
  */
 static bool
-partition_cmp_args(Oid partopcintype, Oid partopfamily,
+partition_cmp_args(PartitionPruneContext *context,
+				   Oid partopcintype, Oid partopfamily,
 				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
 				   bool *result)
 {
@@ -1181,10 +1208,12 @@ partition_cmp_args(Oid partopcintype, Oid partopfamily,
 	 * Try to extract an actual value from each arg.  This may fail if the
 	 * value is unknown in this context, in which case we cannot compare.
 	 */
-	if (!partkey_datum_from_expr(partopcintype, leftarg->value, &left_value))
+	if (!partkey_datum_from_expr(context,
+								 partopcintype, leftarg->value, &left_value))
 		return false;
 
-	if (!partkey_datum_from_expr(partopcintype, rightarg->value, &right_value))
+	if (!partkey_datum_from_expr(context,
+								 partopcintype, rightarg->value, &right_value))
 		return false;
 
 	/*
@@ -1308,12 +1337,14 @@ extract_bounding_datums(PartitionPruneContext *context,
 				case PART_OP_EQUAL:
 					Assert(incl);
 					if (need_next_eq &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1321,7 +1352,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 					}
 
 					if (need_next_min &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1331,7 +1363,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_LESS:
 					if (need_next_max &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1343,7 +1376,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_GREATER:
 					if (need_next_min &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1388,7 +1422,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 			PartClause *pc = (PartClause *) lfirst(lc);
 			Datum	datum;
 
-			if (partkey_datum_from_expr(context->partopcintype[0],
+			if (partkey_datum_from_expr(context,
+										context->partopcintype[0],
 										pc->value, &datum))
 				keys->ne_datums[i++] = datum;
 		}
@@ -1469,7 +1504,8 @@ partition_op_strategy(char part_strategy, PartClause *pc, bool *incl)
  * set.  True is returned otherwise.
  */
 static bool
-partkey_datum_from_expr(Oid partopcintype, Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext *context,
+						Oid partopcintype, Expr *expr, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -1493,25 +1529,193 @@ partkey_datum_from_expr(Oid partopcintype, Expr *expr, Datum *value)
 		if (expr == NULL)
 			return false;
 
-		/*
-		 * Transform into a form that the following code can do something
-		 * useful with.
-		 */
-		expr = evaluate_expr(expr,
-							 exprType((Node *) expr),
-							 exprTypmod((Node *) expr),
-							 exprCollation((Node *) expr));
+		if (context->planstate && context->econtext)
+		{
+			ExprState *exprstate;
+			bool isNull;
+
+			exprstate = ExecInitExpr(expr, context->planstate);
+
+			*value = ExecEvalExprSwitchContext(exprstate,
+											   context->econtext,
+											   &isNull);
+
+			if (isNull)
+				return false;
+
+			return true;
+		}
+		else
+		{
+			/*
+			 * Transform into a form that the following code can do something
+			 * useful with.
+			 */
+			expr = evaluate_expr(expr,
+								 exprType((Node *) expr),
+								 exprTypmod((Node *) expr),
+								 exprCollation((Node *) expr));
+		}
 	}
 
 	/*
 	 * Add more expression types here as needed to support the requirements
 	 * of the higher-level code.
 	 */
-	if (IsA(expr, Const))
+	switch (nodeTag(expr))
 	{
-		*value = ((Const *) expr)->constvalue;
-		return true;
+		case T_Const:
+			*value = ((Const *) expr)->constvalue;
+			return true;
+
+		case T_Param:
+			if (context->planstate && context->econtext)
+			{
+				ExprState *exprstate;
+				bool isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+				*value = ExecEvalExprSwitchContext(exprstate,
+												 context->econtext,
+												 &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+			}
+
+		default:
+			return false;
+	}
+}
+
+/*
+ * make_partition_pruneinfo
+ *		Build PartitionPruneInfo tree to allow the output of
+ *		get_partitions_from_clauses to be translated into
+ *		'subpaths' indexes.  This is required in order to allow
+ *		us to perform any further partition pruning during execution.
+ */
+PartitionPruneInfo *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	PartitionPruneInfo *pinfo;
+	AppendRelInfo	   *appinfo;
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	int					i;
+	int					partidx;
+	int					nparts = rel->nparts;
+
+	check_stack_depth();
+
+	rte = root->simple_rte_array[rel->relid];
+
+	pinfo = makeNode(PartitionPruneInfo);
+	pinfo->relid = rel->relid;
+	pinfo->parentoid = rte->relid;
+	pinfo->prunequal = prunequal;
+	pinfo->allsubnodes = NULL;
+	pinfo->nparts = nparts;
+	pinfo->subnodeindex = (int *) palloc(sizeof(int) * nparts);
+	pinfo->subpartindex = (PartitionPruneInfo **)
+						palloc0(sizeof(PartitionPruneInfo *) * nparts);
+	/*
+	 * -1 represents a partition that has been pruned.  Set them all to this
+	 * initially.  We'll determine the subpath index for the non-pruned
+	 * ones below.
+	 */
+	for (i = 0; i < nparts; i++)
+		pinfo->subnodeindex[i] = -1;
+
+	i = -1;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+
+		i++; /* track subnode index */
+
+		/* Find the AppendRelInfo for the Append child */
+		appinfo = find_childrel_appendrelinfo(root, path->parent);
+
+		/*
+		 * Skip subpaths which belong to relations not directly parented by
+		 * rel.  We'll process any we skip here below when looping through
+		 * partition_rels
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/* Determine the element in part_rel which belongs to this subpath. */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* found it!  Save the subnode index */
+			pinfo->subnodeindex[partidx] = i;
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
 	}
 
-	return false;
+	/*
+	 * Some of the relations returned by get_partitions_from_clauses may be
+	 * other partitioned tables.  Unlike the case above, these won't be
+	 * subpaths of the Append.  To handle these we must create a
+	 * sub-PartitionPruneInfo to allow us to determine if subnodes which
+	 * belong to sub-partitioned tables are required during partition pruning.
+	 */
+	foreach(lc, partition_rels)
+	{
+		Index rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+
+		/*
+		 * partition_rels contains the rti of the base relation being queried.
+		 * We only care about sub-partition parents here, so skip this.
+		 */
+		if (subpart->reloptkind == RELOPT_BASEREL)
+			continue;
+
+		appinfo = find_childrel_appendrelinfo(root, subpart);
+
+		/*
+		 * We only want to deal with sub-partition parents that are directly
+		 * below rel.  We'll deal with any we skip here later in a recursive
+		 * call which is made below.
+		 */
+		if (appinfo->parent_relid != rel->relid)
+			continue;
+
+		/*
+		 * Handle sub-partition parents by building a sub-PartitionPruneInfo.
+		 */
+		for (partidx = 0; partidx < nparts; partidx++)
+		{
+			List *subprunequal;
+
+			if (rel->part_rels[partidx]->relid != appinfo->child_relid)
+				continue;
+
+			/* Adjust the prune qual to be compatible with this subpartition */
+			subprunequal = (List *) adjust_appendrel_attrs(root,
+														(Node *) prunequal,
+														1,
+														&appinfo);
+
+			pinfo->subpartindex[partidx] = make_partition_pruneinfo(root,
+																	subpart,
+															partition_rels,
+																	subpaths,
+																subprunequal);
+			pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, partidx);
+			break;
+		}
+	}
+
+	return pinfo;
 }
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fe3b4582d4..448c05adaa 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,36 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+	pathnode->trypartitionprune = false;
+
+	/*
+	 * When generating an Append path for a partitioned table we'll try to
+	 * enable additional partition pruning at run-time.  Useful pruning quals
+	 * may be in parameterized path quals, so we'll go all the way and
+	 * generate the qual list for the Append's parameterized paths.  We need
+	 * only bother trying this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.
+	 */
+	if (rel->reloptkind == RELOPT_BASEREL && root)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		if (rte->rtekind == RTE_RELATION &&
+			rte->relkind == RELKIND_PARTITIONED_TABLE)
+		{
+			pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+			pathnode->trypartitionprune = true;
+		}
+		else
+			pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																	required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3596,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 6eba13c244..621e8f13b1 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1984,9 +1984,6 @@ set_baserel_partition_key_exprs(Relation relation,
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int			partnatts;
-	int			cnt;
-	List	  **partexprs;
-	ListCell   *lc;
 	Index		varno = rel->relid;
 
 	Assert(IS_SIMPLE_REL(rel) && rel->relid > 0);
@@ -1995,39 +1992,8 @@ set_baserel_partition_key_exprs(Relation relation,
 	Assert(partkey != NULL);
 
 	partnatts = partkey->partnatts;
-	partexprs = (List **) palloc(sizeof(List *) * partnatts);
-	lc = list_head(partkey->partexprs);
 
-	for (cnt = 0; cnt < partnatts; cnt++)
-	{
-		Expr	   *partexpr;
-		AttrNumber	attno = partkey->partattrs[cnt];
-
-		if (attno != InvalidAttrNumber)
-		{
-			/* Single column partition key is stored as a Var node. */
-			Assert(attno > 0);
-
-			partexpr = (Expr *) makeVar(varno, attno,
-										partkey->parttypid[cnt],
-										partkey->parttypmod[cnt],
-										partkey->parttypcoll[cnt], 0);
-		}
-		else
-		{
-			if (lc == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			/* Re-stamp the expression with given varno. */
-			partexpr = (Expr *) copyObject(lfirst(lc));
-			ChangeVarNodes((Node *) partexpr, 1, varno, 0);
-			lc = lnext(lc);
-		}
-
-		partexprs[cnt] = list_make1(partexpr);
-	}
-
-	rel->partexprs = partexprs;
+	rel->partexprs = build_partition_expressions(partkey, varno);
 
 	/*
 	 * A base relation can not have nullable partition key expressions. We
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 4e9281d3d5..f63eb70335 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -67,6 +68,12 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo	boundinfo;
+
+	PlanState	*planstate;
+	ExprContext *econtext;
+
+	/* ParamIds of clauses being used to determine partitions */
+	Bitmapset *paramids;
 } PartitionPruneContext;
 
 /*
@@ -162,4 +169,6 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 extern Bitmapset *get_partitions_for_keys(PartitionPruneContext *context,
 						PartScanKeyInfo *keys);
 
+extern List **build_partition_expressions(PartitionKey partkey, Index varno);
+
 #endif							/* PARTITION_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index a953820f43..7db3a79e71 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1010,11 +1010,14 @@ typedef struct ModifyTableState
  *
  *		nplans			how many plans are in the array
  *		whichplan		which plan is being executed (0 .. n-1)
+ *		valid_subplans	for runtime pruning, valid appendplans indexes to scan
  * ----------------
  */
 
 struct AppendState;
 typedef struct AppendState AppendState;
+struct PartitionPruneContextCache;
+typedef struct PartitionPruneContextCache PartitionPruneContextCache;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
 
@@ -1026,6 +1029,11 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Bitmapset  *as_valid_subplans; /* mask of non-pruned subplans */
+	Bitmapset  *part_prune_params; /* ParamIds useful for partition pruning */
+	PartitionPruneInfo *part_prune_info; /* details for partition pruning */
+	PartitionPruneContextCache *contextcache; /* cache of prune contexts */
+	MemoryContext prune_context; /* used when calling planner pruning code */
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c097da6425..d693f3711c 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f2e19eae68..2264d54c2a 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -250,6 +250,13 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning. This allows translation
+	 * from partition index into subplan indexes.  This is set to NULL when
+	 * run-time subplan pruning is disabled.
+	 */
+	PartitionPruneInfo *part_prune_info;
 } Append;
 
 /* ----------------
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d75af..ffb6daffdd 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,27 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Allows pruning of Append subplans
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into an Append node's subplan index.  This structure is used to
+ * recursively search for all subplan nodes when there are sub-partitioned
+ * tables in the Append plan.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	int			relid;		/* relation index of parent partition rel */
+	Oid			parentoid;	/* Oid of parent partition rel */
+	List	   *prunequal;	/* qual list for pruning partitions */
+	Bitmapset  *allsubnodes;	/* All subnode indexes at this level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+
+	/* sub-PartitionPruneInfo indexed by partition id */
+	struct PartitionPruneInfo **subpartindex;
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 34d79f284b..dbf280cc6e 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1304,6 +1304,8 @@ typedef struct AppendPath
 
 	/* Index of first partial path in subpaths */
 	int			first_partial_path;
+
+	bool		trypartitionprune; /* Attempt to enable partition pruning? */
 } AppendPath;
 
 #define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 3c2f54964b..bedffc4189 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -80,6 +80,9 @@ extern void CommuteRowCompareExpr(RowCompareExpr *clause);
 
 extern Node *eval_const_expressions(PlannerInfo *root, Node *node);
 
+extern Node *eval_const_expressions_from_list(ParamListInfo prmlist,
+								 Node *node);
+
 extern Node *estimate_expression_value(PlannerInfo *root, Node *node);
 
 extern Query *inline_set_returning_function(PlannerInfo *root,
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index b654691e9b..9d62e13c72 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -68,4 +68,9 @@ extern PartitionClauseInfo *generate_partition_clauses(
 extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context,
 							PartitionClauseInfo *partclauseinfo);
 
+extern PartitionPruneInfo *make_partition_pruneinfo(PlannerInfo *root,
+						 RelOptInfo *rel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index ef7173fbf8..bde18582e9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 948cad4c3d..424b3a7b1b 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1475,3 +1475,1118 @@ explain (costs off) select * from like_op_noprune where a like '%BC';
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+deallocate q1;
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 08fc2dbc21..73b4d109a5 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -253,3 +253,315 @@ create table like_op_noprune2 partition of like_op_noprune for values in ('BCD')
 explain (costs off) select * from like_op_noprune where a like '%BC';
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
-- 
2.11.0

#90David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#89)
Re: [HACKERS] Runtime Partition Pruning

On 27 February 2018 at 22:39, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:

I've incorporated portions of 0002 and 0003 into my patch on the other
thread (v34) posted at [1]. That is, mostly the changes around handling
OR clauses and interface changes resulting from it.

Thanks. I was just in the middle of swapping the order of the patches
so that the OR clause patch was directly based on yours.

Attached are revised version of your patches after the aforementioned
rearrangements. Note that after I took out the optimizer portion of the
0003 patch to incorporate it into my patch (OR clause processing bits),
not much was left in it, so I squashed it into 0002. So there are only
0001 and 0002.

I've locally got a patch which is significantly different to the v12
patch which moves lots of code into nodePartition.c and fixes up the
missing node read/write functions too.

As a review comment on 0002, I think trypartitionprune is better written
as try_partition_prune.

That no longer exists in the new version... Will post soonish, just
need to base it all on your v34 [1]/messages/by-id/158f04ce-9deb-0457-ddcc-78fb73db4ebc@lab.ntt.co.jp now! :)

[1]: /messages/by-id/158f04ce-9deb-0457-ddcc-78fb73db4ebc@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#91David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#90)
5 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

I've attached v14 of the patch.

This is based on Amit's faster partition pruning patch v35 [1]/messages/by-id/0f96dd16-f5d5-7301-4ddf-858d41a6cbe3@lab.ntt.co.jp

There's quite a bit of code churn in this version from the last
version. I've now moved most of the run-time pruning code into
execPartition.c and aimed to make it more generic to apply to node
types other than Append. I also had to make a few changes to the
PartitionPruneInfo node type so that it could get some support in
readfuncs.c and outfuncs.c, which I had previously missed. As proof
that the code in execPartition.c is fairly generic and applies to any
subnode type that supports a List of subnodes, I went ahead and wrote
a small additional patch to add support for run-time pruning for
MergeAppend. This just takes an extra 100 lines of code.

I've also split the patch out a bit more into logical parts in the
hope it makes things easier to review.

[1]: /messages/by-id/0f96dd16-f5d5-7301-4ddf-858d41a6cbe3@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v14-0001-Add-bms_prev_member-function.patchapplication/octet-stream; name=v14-0001-Add-bms_prev_member-function.patchDownload
From 5a9ffb3be4d2fb969f6000fa25cd07938ab4a1cf Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Thu, 1 Mar 2018 22:59:26 +1300
Subject: [PATCH v14 1/5] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a..3578c8f 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920..b6f1a9e 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
1.9.5.msysgit.1

v14-0002-Extract-code-to-build-partition-expressions-into.patchapplication/octet-stream; name=v14-0002-Extract-code-to-build-partition-expressions-into.patchDownload
From 7ae7551bb3de843d2b9b34b04751c5974f56b52e Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Thu, 1 Mar 2018 22:59:58 +1300
Subject: [PATCH v14 2/5] Extract code to build partition expressions into a
 reusable function

---
 src/backend/catalog/partition.c      | 50 ++++++++++++++++++++++++++++++++++++
 src/backend/optimizer/util/plancat.c | 36 +-------------------------
 src/include/catalog/partition.h      |  2 ++
 3 files changed, 53 insertions(+), 35 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 021170a..311d0d3 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -4200,3 +4200,53 @@ satisfies_hash_partition(PG_FUNCTION_ARGS)
 
 	PG_RETURN_BOOL(rowHash % modulus == remainder);
 }
+
+/*
+ * build_partition_expressions
+ *		Build a list of partition key expressions.  Plain attibute keys will
+ *		be build as Vars with the varno set to 'varno'.
+ */
+List **
+build_partition_expressions(PartitionKey partkey, Index varno)
+{
+	int			partnatts;
+	int			cnt;
+	List	  **partexprs;
+	ListCell   *lc;
+
+	partnatts = partkey->partnatts;
+	partexprs = (List **) palloc(sizeof(List *) * partnatts);
+	lc = list_head(partkey->partexprs);
+
+	for (cnt = 0; cnt < partnatts; cnt++)
+	{
+		Expr	   *partexpr;
+		AttrNumber	attno = partkey->partattrs[cnt];
+
+		if (attno != InvalidAttrNumber)
+		{
+			/* Single column partition key is stored as a Var node. */
+			Assert(attno > 0);
+
+			partexpr = (Expr *) makeVar(varno, attno,
+										partkey->parttypid[cnt],
+										partkey->parttypmod[cnt],
+										partkey->parttypcoll[cnt], 0);
+		}
+		else
+		{
+			if (lc == NULL)
+				elog(ERROR, "wrong number of partition key expressions");
+
+			/* Re-stamp the expression with given varno. */
+			partexpr = (Expr *) copyObject(lfirst(lc));
+			if (varno != 1)
+				ChangeVarNodes((Node *) partexpr, 1, varno, 0);
+			lc = lnext(lc);
+		}
+
+		partexprs[cnt] = list_make1(partexpr);
+	}
+
+	return partexprs;
+}
diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c
index 4dcca8b..03c1c35 100644
--- a/src/backend/optimizer/util/plancat.c
+++ b/src/backend/optimizer/util/plancat.c
@@ -1993,9 +1993,6 @@ set_baserel_partition_key_exprs(Relation relation,
 {
 	PartitionKey partkey = RelationGetPartitionKey(relation);
 	int			partnatts;
-	int			cnt;
-	List	  **partexprs;
-	ListCell   *lc;
 	Index		varno = rel->relid;
 
 	Assert(IS_SIMPLE_REL(rel) && rel->relid > 0);
@@ -2004,39 +2001,8 @@ set_baserel_partition_key_exprs(Relation relation,
 	Assert(partkey != NULL);
 
 	partnatts = partkey->partnatts;
-	partexprs = (List **) palloc(sizeof(List *) * partnatts);
-	lc = list_head(partkey->partexprs);
-
-	for (cnt = 0; cnt < partnatts; cnt++)
-	{
-		Expr	   *partexpr;
-		AttrNumber	attno = partkey->partattrs[cnt];
-
-		if (attno != InvalidAttrNumber)
-		{
-			/* Single column partition key is stored as a Var node. */
-			Assert(attno > 0);
-
-			partexpr = (Expr *) makeVar(varno, attno,
-										partkey->parttypid[cnt],
-										partkey->parttypmod[cnt],
-										partkey->parttypcoll[cnt], 0);
-		}
-		else
-		{
-			if (lc == NULL)
-				elog(ERROR, "wrong number of partition key expressions");
-
-			/* Re-stamp the expression with given varno. */
-			partexpr = (Expr *) copyObject(lfirst(lc));
-			ChangeVarNodes((Node *) partexpr, 1, varno, 0);
-			lc = lnext(lc);
-		}
-
-		partexprs[cnt] = list_make1(partexpr);
-	}
 
-	rel->partexprs = partexprs;
+	rel->partexprs = build_partition_expressions(partkey, varno);
 
 	/*
 	 * A base relation can not have nullable partition key expressions. We
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 4e9281d..8a487e0 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -162,4 +162,6 @@ extern int get_partition_for_tuple(Relation relation, Datum *values,
 extern Bitmapset *get_partitions_for_keys(PartitionPruneContext *context,
 						PartScanKeyInfo *keys);
 
+extern List **build_partition_expressions(PartitionKey partkey, Index varno);
+
 #endif							/* PARTITION_H */
-- 
1.9.5.msysgit.1

v14-0003-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v14-0003-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From da26ad7bd253fdfbc5cda9d304cc0da8a0f2734a Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Thu, 1 Mar 2018 23:02:26 +1300
Subject: [PATCH v14 3/5] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/executor/execPartition.c   | 234 ++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c          |  19 +++
 src/backend/nodes/outfuncs.c           |  25 +++
 src/backend/nodes/readfuncs.c          |  18 +++
 src/backend/optimizer/util/partprune.c | 269 ++++++++++++++++++++++++++++-----
 src/include/catalog/partition.h        |  10 ++
 src/include/executor/execPartition.h   |  67 ++++++++
 src/include/nodes/nodes.h              |   1 +
 src/include/nodes/primnodes.h          |  21 +++
 src/include/optimizer/partprune.h      |   4 +
 10 files changed, 634 insertions(+), 34 deletions(-)

diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 54efc9e..9a87245 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -19,6 +19,7 @@
 #include "executor/executor.h"
 #include "mb/pg_wchar.h"
 #include "miscadmin.h"
+#include "optimizer/partprune.h"
 #include "utils/lsyscache.h"
 #include "utils/rls.h"
 #include "utils/ruleutils.h"
@@ -36,6 +37,9 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 Datum *values,
 									 bool *isnull,
 									 int maxfieldlen);
+static void find_matching_subplans_recurse(
+							   PartitionedRelPruning *partrelprune,
+							   Bitmapset **validsubplans);
 
 /*
  * ExecSetupPartitionTupleRouting - sets up information needed during
@@ -946,3 +950,233 @@ ExecBuildSlotPartitionKeyDescription(Relation rel,
 
 	return buf.data;
 }
+
+/*
+ * ExecSetupPartitionPruning
+ *
+ * Setup the required data structure for calling ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we pre-process the 'prunequal' of each
+ * PartitionPruneInfo and also extract any parameter IDs of Params which
+ * can be matched to the partition key.  We also build a PartitionPruneContext
+ * for each item in the List.  These are reusable and need only be setup once.
+ *
+ * It is possible that no params can be matched to for one, or even all
+ * partition parents.  For the latter case, there is not much use in calling
+ * ExecFindMatchingSubPlans() on the return value of this function as it will
+ * always just return all possible subnodes.  The param IDs of all parameters
+ * matching partition keys in the entire partition hierarchy are stored in
+ * the return value's 'prune_param_ids' field.  Callers may wish to check
+ * that this is not an empty set before attempting to further narrow the list
+ * of subplans to scan.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionedRelPruning *partrelprunes;
+	PartitionPruning *partprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	partprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionedRelPruning *)
+					palloc(sizeof(PartitionedRelPruning) *
+						   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	partprune->partition_rel_pruning = &partrelprunes[0];
+	partprune->prune_param_ids = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	partprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+													 "Partition Prune",
+													 ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionedRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc		partdesc;
+		Relation			rel;
+		PartitionKey		partkey;
+		List			  **partexprs;
+		int					partnatts;
+		int		j;
+
+		partrelprune->relid = pinfo->relid;
+		partrelprune->allsubnodes = pinfo->allsubnodes;
+		partrelprune->nparts = pinfo->nparts;
+		partrelprune->subnodeindex = pinfo->subnodeindex;
+		partrelprune->subpartprune = palloc(sizeof(PartitionedRelPruning *) *
+										 pinfo->nparts);
+
+		for (j = 0; j < pinfo->nparts; j++)
+		{
+			int subpartidx = pinfo->subpartindex[j];
+
+			Assert(subpartidx < list_length(partitionpruneinfo));
+
+			if (subpartidx >= 0)
+				partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
+			else
+				partrelprune->subpartprune[j] = NULL;
+		}
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->relid = pinfo->relid;
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+		partexprs = build_partition_expressions(partkey, pinfo->relid);
+		context->partkeys = (Expr **) palloc(sizeof(Expr *) * partnatts);
+
+		for (j = 0; j < partnatts; j++)
+			context->partkeys[j] = (Expr *) linitial(partexprs[j]);
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		if (OidIsValid(get_default_oid_from_partdesc(partdesc)))
+			context->has_default_part = true;
+		else
+			context->has_default_part = false;
+
+		context->partition_qual = RelationGetPartitionQual(rel);
+
+		context->planstate = planstate;
+		context->paramids = NULL;
+
+		partrelprune->partclauseinfo = generate_partition_clauses(context,
+															pinfo->prunequal);
+
+		partprune->prune_param_ids =
+								bms_add_members(partprune->prune_param_ids,
+												context->paramids);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	return partprune;
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to scan based on the
+ *		details stored in 'partprune'.  All subplans which provably cannot
+ *		possibly have matching records are eliminated and the indexes of the
+ *		remaining set are returned in the form of a Bitmapset.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *partprune)
+{
+	PartitionedRelPruning *partrelprune;
+	MemoryContext		oldcontext;
+	Bitmapset *result = NULL;
+
+	partrelprune = partprune->partition_rel_pruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	find_matching_subplans_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_matching_subplans_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans.
+ */
+static void
+find_matching_subplans_recurse(PartitionedRelPruning *partrelprune,
+							   Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	PartitionClauseInfo *partclauseinfo;
+	Bitmapset	   *partset;
+	int			i;
+
+	check_stack_depth();
+
+	partclauseinfo = partrelprune->partclauseinfo;
+
+	/*
+	 * Detect if any impossibilities were discovered during
+	 * generate_partition_clauses
+	 */
+	if (partclauseinfo->constfalse)
+	{
+		bms_free(*validsubplans);
+		*validsubplans = NULL;
+		return;
+	}
+
+	/*
+	 * 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 subpartitions as we may find their partitions keys match some
+	 * params.
+	 */
+	if (!bms_is_empty(context->paramids))
+		partset = get_partitions_from_clauses(context, partclauseinfo);
+	else
+		partset = partrelprune->allsubnodes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_matching_subplans_recurse(partrelprune->subpartprune[i],
+										   validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
\ No newline at end of file
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 169c697..2d6b943 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2132,6 +2132,22 @@ _copyOnConflictExpr(const OnConflictExpr *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(relid);
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(prunequal);
+	COPY_BITMAPSET_FIELD(allsubnodes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(int));
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5028,6 +5044,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index fe309a6..6fd547c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1707,6 +1707,28 @@ _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_INT_FIELD(relid);
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(prunequal);
+	WRITE_BITMAPSET_FIELD(allsubnodes);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnodeindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnodeindex[i]);
+
+	appendStringInfoString(str, " :subpartindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpartindex[i]);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3922,6 +3944,9 @@ outNode(StringInfo str, const void *obj)
 			case T_OnConflictExpr:
 				_outOnConflictExpr(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 068db35..7bb92d4 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1328,6 +1328,22 @@ _readOnConflictExpr(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_INT_FIELD(relid);
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(prunequal);
+	READ_BITMAPSET_FIELD(allsubnodes);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnodeindex, local_node->nparts);
+	READ_INT_ARRAY(subpartindex, local_node->nparts);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2571,6 +2587,8 @@ parseNodeString(void)
 		return_value = _readFromExpr();
 	else if (MATCH("ONCONFLICTEXPR", 14))
 		return_value = _readOnConflictExpr();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 70a215f..096f2d4 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -38,6 +38,11 @@
  * while also taking into account strategies of the operators in the matched
  * clauses.
  *
+ * make_partition_pruneinfo()
+ *
+ * Generates a List of PartitionPruneInfo nodes for use in the executor to
+ * allow it to perform partition pruning during execution.
+ *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
  *
@@ -53,10 +58,12 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/executor.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -104,16 +111,16 @@ static Bitmapset *get_partitions_from_or_args(PartitionPruneContext *context,
 static void remove_redundant_clauses(PartitionPruneContext *context,
 						 PartitionClauseInfo *partclauseinfo,
 						 List **minimalclauses);
-static bool partition_cmp_args(Oid partopcintype, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result);
+static bool partition_cmp_args(PartitionPruneContext *context,
+				   Oid partopcintype, Oid partopfamily, PartClause *pc,
+				   PartClause *leftarg, PartClause *rightarg, bool *result);
 static bool extract_bounding_datums(PartitionPruneContext *context,
 						PartitionClauseInfo *clauseinfo,
 						List **minimalclauses, PartScanKeyInfo *keys);
 static PartOpStrategy partition_op_strategy(char part_strategy,
 					PartClause *pc, bool *incl);
-static bool partkey_datum_from_expr(Oid partopcintype, Expr *expr,
-					Datum *value);
+static bool partkey_datum_from_expr(PartitionPruneContext *context,
+					Oid partopcintype, Expr *expr, Datum *value);
 
 /*
  * prune_append_rel_partitions
@@ -158,6 +165,8 @@ prune_append_rel_partitions(PlannerInfo *root, RelOptInfo *rel)
 		context.boundinfo = rel->boundinfo;
 		context.has_default_part = rel->has_default_part;
 		context.partition_qual = rel->partition_qual;
+		context.planstate = NULL;
+		context.paramids = NULL;
 
 		/* process clauses */
 		partclauseinfo = generate_partition_clauses(&context, clauses);
@@ -534,6 +543,10 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 				pc->inputcollid = opclause->inputcollid;
 				pc->value = valueexpr;
 
+				if (IsA(valueexpr, Param))
+					context->paramids = bms_add_member(context->paramids,
+											((Param *) valueexpr)->paramid);
+
 				/*
 				 * We don't turn a <> operator clause into a key right away.
 				 * Instead, the caller will hand over such clauses to
@@ -695,6 +708,11 @@ extract_partition_clauses(PartitionPruneContext *context, List *clauses)
 														 leftop, rightop,
 														 InvalidOid,
 														 saop_coll);
+
+					if (IsA(rightop, Param))
+						context->paramids = bms_add_member(context->paramids,
+												((Param *) rightop)->paramid);
+
 					elem_clauses = lappend(elem_clauses, elem_clause);
 				}
 
@@ -966,7 +984,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				if (hash_clause == NULL)
 					hash_clause = pc;
 				/* check if another clause would contradict the one we have */
-				else if (partition_cmp_args(context->partopcintype[i],
+				else if (partition_cmp_args(context,
+											context->partopcintype[i],
 											context->partopfamily[i],
 											pc, pc, hash_clause,
 											&test_result))
@@ -1023,7 +1042,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * then because 7 < 5 is false, we leave a < 5 where it is and
 				 * effectively discard a < 7 as being redundant.
 				 */
-				if (partition_cmp_args(context->partopcintype[i],
+				if (partition_cmp_args(context,
+									   context->partopcintype[i],
 									   context->partopfamily[i],
 									   pc, pc, btree_clauses[s],
 									   &test_result))
@@ -1080,7 +1100,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 				 * eq clause is a = 3, then because 3 < 5, we no longer need
 				 * a < 5, because a = 3 is more restrictive.
 				 */
-				if (partition_cmp_args(context->partopcintype[i],
+				if (partition_cmp_args(context,
+									   context->partopcintype[i],
 									   context->partopfamily[i],
 									   chk, eq, chk,
 									   &test_result))
@@ -1111,7 +1132,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *lt = btree_clauses[BTLessStrategyNumber - 1],
 					   *le = btree_clauses[BTLessEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->partopcintype[i],
+			if (partition_cmp_args(context,
+								   context->partopcintype[i],
 								   context->partopfamily[i],
 								   le, lt, le,
 								   &test_result))
@@ -1130,7 +1152,8 @@ remove_redundant_clauses(PartitionPruneContext *context,
 			PartClause *gt = btree_clauses[BTGreaterStrategyNumber - 1],
 					   *ge = btree_clauses[BTGreaterEqualStrategyNumber - 1];
 
-			if (partition_cmp_args(context->partopcintype[i],
+			if (partition_cmp_args(context,
+								   context->partopcintype[i],
 								   context->partopfamily[i],
 								   ge, gt, ge,
 								   &test_result))
@@ -1168,9 +1191,9 @@ remove_redundant_clauses(PartitionPruneContext *context,
  * incompatible with the operator.
  */
 static bool
-partition_cmp_args(Oid partopcintype, Oid partopfamily,
-				   PartClause *pc, PartClause *leftarg, PartClause *rightarg,
-				   bool *result)
+partition_cmp_args(PartitionPruneContext *context, Oid partopcintype,
+				   Oid partopfamily, PartClause *pc, PartClause *leftarg,
+				   PartClause *rightarg, bool *result)
 {
 	Datum	left_value;
 	Datum	right_value;
@@ -1181,10 +1204,12 @@ partition_cmp_args(Oid partopcintype, Oid partopfamily,
 	 * Try to extract an actual value from each arg.  This may fail if the
 	 * value is unknown in this context, in which case we cannot compare.
 	 */
-	if (!partkey_datum_from_expr(partopcintype, leftarg->value, &left_value))
+	if (!partkey_datum_from_expr(context, partopcintype, leftarg->value,
+								 &left_value))
 		return false;
 
-	if (!partkey_datum_from_expr(partopcintype, rightarg->value, &right_value))
+	if (!partkey_datum_from_expr(context, partopcintype, rightarg->value,
+								 &right_value))
 		return false;
 
 	/*
@@ -1308,12 +1333,14 @@ extract_bounding_datums(PartitionPruneContext *context,
 				case PART_OP_EQUAL:
 					Assert(incl);
 					if (need_next_eq &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->eqkeys[i]))
 						keys->n_eqkeys++;
 
 					if (need_next_max &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1321,7 +1348,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 					}
 
 					if (need_next_min &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1331,7 +1359,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_LESS:
 					if (need_next_max &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->maxkeys[i]))
 					{
 						keys->n_maxkeys++;
@@ -1343,7 +1372,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 
 				case PART_OP_GREATER:
 					if (need_next_min &&
-						partkey_datum_from_expr(context->partopcintype[i],
+						partkey_datum_from_expr(context,
+												context->partopcintype[i],
 												value, &keys->minkeys[i]))
 					{
 						keys->n_minkeys++;
@@ -1388,7 +1418,8 @@ extract_bounding_datums(PartitionPruneContext *context,
 			PartClause *pc = (PartClause *) lfirst(lc);
 			Datum	datum;
 
-			if (partkey_datum_from_expr(context->partopcintype[0],
+			if (partkey_datum_from_expr(context,
+										context->partopcintype[0],
 										pc->value, &datum))
 				keys->ne_datums[i++] = datum;
 		}
@@ -1469,7 +1500,8 @@ partition_op_strategy(char part_strategy, PartClause *pc, bool *incl)
  * set.  True is returned otherwise.
  */
 static bool
-partkey_datum_from_expr(Oid partopcintype, Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext *context, Oid partopcintype,
+						Expr *expr, Datum *value)
 {
 	Oid		exprtype = exprType((Node *) expr);
 
@@ -1493,25 +1525,194 @@ partkey_datum_from_expr(Oid partopcintype, Expr *expr, Datum *value)
 		if (expr == NULL)
 			return false;
 
-		/*
-		 * Transform into a form that the following code can do something
-		 * useful with.
-		 */
-		expr = evaluate_expr(expr,
-							 exprType((Node *) expr),
-							 exprTypmod((Node *) expr),
-							 exprCollation((Node *) expr));
+		if (context->planstate)
+		{
+			ExprState *exprstate;
+			bool isNull;
+
+			exprstate = ExecInitExpr(expr, context->planstate);
+
+			*value = ExecEvalExprSwitchContext(exprstate,
+										context->planstate->ps_ExprContext,
+											   &isNull);
+
+			if (isNull)
+				return false;
+
+			return true;
+		}
+		else
+		{
+			/*
+			 * Transform into a form that the following code can do something
+			 * useful with.
+			 */
+			expr = evaluate_expr(expr,
+								 exprType((Node *) expr),
+								 exprTypmod((Node *) expr),
+								 exprCollation((Node *) expr));
+		}
 	}
 
 	/*
 	 * Add more expression types here as needed to support the requirements
 	 * of the higher-level code.
 	 */
-	if (IsA(expr, Const))
+	switch (nodeTag(expr))
 	{
-		*value = ((Const *) expr)->constvalue;
-		return true;
+		case T_Const:
+			*value = ((Const *) expr)->constvalue;
+			return true;
+
+		case T_Param:
+			if (context->planstate)
+			{
+				ExprState *exprstate;
+				bool isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+				*value = ExecEvalExprSwitchContext(exprstate,
+												 context->planstate->ps_ExprContext,
+												 &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+			}
+
+		default:
+			return false;
 	}
+}
 
-	return false;
+/*
+ * make_partition_pruneinfo
+ *		Build a List of PartitionPruneInfos, one for each 'partitioned_rel'.
+ *
+ * Here we index the subpaths by partition index so that we're able to
+ * translate the output of get_partitions_from_clauses into subpath indexes
+ * to possibly allow for further partition pruning to be performed during
+ * execution.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal)
+{
+	RangeTblEntry	   *rte;
+	ListCell		   *lc;
+	List			   *pinfolist = NIL;
+	int				   *allsubnodeindex;
+	int				   *allsubpartindex;
+	int					i;
+
+	Assert(parentrel->reloptkind == RELOPT_BASEREL);
+
+	/*
+	 * Allocate two arrays, one to allow quick lookups of the 'subpaths' index
+	 * of a relation by relid and another to lookup the 'partitioned_rel'
+	 * index by relid.
+	 */
+	allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
+	allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);
+
+	/* Initialize to -1 to indicate the rel was not found */
+	for (i = 0; i < root->simple_rel_array_size; i++)
+	{
+		allsubnodeindex[i] = -1;
+		allsubpartindex[i] = -1;
+	}
+
+	/*
+	 * Now loop over each subpath and fill in the index of the subpath for the
+	 * subpath's relid.
+	 */
+	i = 0;
+	foreach(lc, subpaths)
+	{
+		Path *path = (Path *) lfirst(lc);
+		RelOptInfo *pathrel = path->parent;
+
+		Assert(IS_SIMPLE_REL(pathrel));
+		Assert(pathrel->relid < root->simple_rel_array_size);
+
+		allsubnodeindex[pathrel->relid] = i;
+		i++;
+	}
+
+	/* Likewise for the partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index	rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		allsubpartindex[rti] = i;
+		i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		int			nparts = subpart->nparts;
+		int		   *subnodeindex;
+		int		   *subpartindex;
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->relid = subpart->relid;
+		pinfo->reloid = rte->relid;
+
+		/* prunequal will only require translation for sub-partitions */
+		if (subpart == parentrel)
+			pinfo->prunequal = prunequal;
+		else
+			pinfo->prunequal = (List *)
+						adjust_appendrel_attrs_multilevel(root,
+														  (Node *) prunequal,
+														  subpart->relids,
+														  parentrel->relids);
+		pinfo->allsubnodes = NULL;
+		pinfo->nparts = nparts;
+		pinfo->subnodeindex = subnodeindex = palloc(nparts * sizeof(int));
+		pinfo->subpartindex = subpartindex = palloc(nparts * sizeof(int));
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present
+		 * in the subpaths List will be set to -1, and any subpartitioned
+		 * table which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = allsubnodeindex[partrel->relid];
+			int			subpartidx = allsubpartindex[partrel->relid];
+
+			subnodeindex[i] = subnodeidx;
+			subpartindex[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				pinfo->allsubnodes = bms_add_member(pinfo->allsubnodes, i);
+		}
+
+		pinfolist = lappend(pinfolist, pinfo);
+	}
+
+	pfree(allsubnodeindex);
+	pfree(allsubpartindex);
+
+	return pinfolist;
 }
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8a487e0..88b934e 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -67,6 +68,15 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo	boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow
+	 * resolution of Param values.
+	 */
+	PlanState	*planstate;
+
+	/* ParamIds of clauses matching the partition key */
+	Bitmapset *paramids;
 } PartitionPruneContext;
 
 /*
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 03a599a..6fdd514 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "optimizer/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,69 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionedRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows partprune.c's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the quals stored in 'partclauseinfo' will be found within.
+ *
+ * relid						Relation index of the partitioned table that
+ *								this information belongs to.
+ * nparts						The number of partitions which belong to this
+ *								partitioned relation. Also defines the size of
+ *								the 'subnodeindex' and 'subpartprune' arrays.
+ * subnodeindex					An array of nparts containing the subnode
+ *								index which matches this partition index, or
+ *								-1 if there is no match.
+ * subpartprune					An array of nparts containing the
+ *								PartitionedRelPruning details this partition
+ *								index for sub-partitioned tables.
+ * allsubnodes					A Bitmapset of all subnode indexes which
+ *								belong to this partition.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * partclauseinfo				Contains clauses pre-matched to the partition
+ *								key ready to be used for partition pruning.
+ *-----------------------
+ */
+typedef struct PartitionedRelPruning
+{
+	int			relid;
+	int			nparts;
+	int		   *subnodeindex;
+	struct PartitionedRelPruning **subpartprune;
+	Bitmapset  *allsubnodes;
+	PartitionPruneContext context;
+	PartitionClauseInfo *partclauseinfo;
+} PartitionedRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionedRelPruning
+ * structs and also stores all Param IDs which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partition_rel_pruning		PartitionedRelPruning for the node's target
+ *								partitioned relation.
+ * prune_param_ids				All Param IDs which were found to match a
+ *								partition key in each of the contained
+ *								PartitionedRelPruning structs.
+ * prune_context				A memory context which can be used to call
+ *								the query planner's partition prune functions.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionedRelPruning *partition_rel_pruning;
+	Bitmapset *prune_param_ids;
+	MemoryContext prune_context;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -126,5 +190,8 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index c097da6..d693f37 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -191,6 +191,7 @@ typedef enum NodeTag
 	T_FromExpr,
 	T_OnConflictExpr,
 	T_IntoClause,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 1b4b0d7..bbf3c68 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1506,4 +1506,25 @@ typedef struct OnConflictExpr
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	int			relid;		/* relation index of parent partition rel */
+	Oid			reloid;		/* Oid of partition rel */
+	List	   *prunequal;	/* qual list for pruning partitions */
+	Bitmapset  *allsubnodes;	/* All subnode indexes at this level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+	int		   *subpartindex;	/* subpart index indexed by partition id */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index b654691..ec96da4 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -68,4 +68,8 @@ extern PartitionClauseInfo *generate_partition_clauses(
 extern Bitmapset *get_partitions_from_clauses(PartitionPruneContext *context,
 							PartitionClauseInfo *partclauseinfo);
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, RelOptInfo *parentrel,
+						 List *partition_rels, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
-- 
1.9.5.msysgit.1

v14-0004-Allow-Append-subnodes-to-be-pruned-during-execut.patchapplication/octet-stream; name=v14-0004-Allow-Append-subnodes-to-be-pruned-during-execut.patchDownload
From dba47623ca2088ae0dbbd81f3fb497092548ff5d Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Thu, 1 Mar 2018 23:03:16 +1300
Subject: [PATCH v14 4/5] Allow Append subnodes to be pruned during execution

Support partition pruning of Append subnodes which cannot possible contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Unlike the case of partition elimination during query planning, when the
elimination takes place during execution, the Append's subnodes are still
visible in the EXPLAIN output.  In order to determine if pruning has actually
taken place the EXPLAIN ANALYZE must be viewed.  If a certain Append subplan
was never executed due to elimination of the partition then the execution
timing area will state "(never executed)".  Whereas, if, for example in the
case of parameterized nested loops, the number of loops stated in the EXPLAIN
ANALYZE output for certain subplans may appear lower than others due to the
subplan having been scanned fewer times.  This is due to the list of matching
subnodes having to be evaluated whenever a parameter which was found to match
the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  154 +++-
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    3 +-
 src/backend/optimizer/prep/prepunion.c        |    4 +-
 src/backend/optimizer/util/pathnode.c         |   30 +-
 src/include/nodes/execnodes.h                 |   12 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1115 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  312 +++++++
 15 files changed, 1647 insertions(+), 51 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d6..63ba402 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -149,18 +151,30 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
+	 * create expression context for node, if required for partition pruning.
 	 */
+	if (node->part_prune_infos != NIL)
+	{
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		appendstate->partition_pruning =
+								ExecSetupPartitionPruning(&appendstate->ps,
+														  node->part_prune_infos);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +265,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->prune_param_ids))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +297,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +387,36 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -415,6 +449,23 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +518,18 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +595,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 2d6b943..65e02a2 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -244,6 +244,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 6fd547c..a827cc6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -401,6 +401,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 7bb92d4..fa869a0 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1623,6 +1623,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 08570ce..5bcb199 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1587,7 +1587,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1629,8 +1629,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1678,7 +1678,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1734,7 +1734,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2000,7 +2000,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3..2e289d4 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 9ae1bf3..1a788eb 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -204,7 +205,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1022,6 +1023,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1059,6 +1062,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root, rel,
+											best_path->partitioned_rels,
+											best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1067,7 +1101,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5315,7 +5350,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5327,7 +5363,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4b5713f..9351e0c 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3736,7 +3736,8 @@ create_grouping_paths(PlannerInfo *root,
 				paths = lappend(paths, path);
 			}
 			path = (Path *)
-				create_append_path(grouped_rel,
+				create_append_path(root,
+								   grouped_rel,
 								   paths,
 								   NIL,
 								   NULL,
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index f01119e..146e202 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -593,7 +593,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 	/* We have to manually jam the right tlist into the path; ick */
 	path->pathtarget = create_pathtarget(root, tlist);
@@ -705,7 +705,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* We have to manually jam the right tlist into the path; ick */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fe3b458..a8e5a87 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,29 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+	{
+		RangeTblEntry *rte = planner_rt_fetch(rel->relid, root);
+
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+	}
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3589,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index a953820..4141f34 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1008,13 +1008,19 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		as_valid_subplans	for runtime pruning, valid appendplans indexes to
+ *							scan.
  * ----------------
  */
 
 struct AppendState;
 typedef struct AppendState AppendState;
+struct PartitionPruning;
+typedef struct PartitionPruning PartitionPruning;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
 
@@ -1026,6 +1032,8 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	PartitionPruning *partition_pruning;
+	Bitmapset   *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f2e19ea..c94937edae 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -250,6 +250,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index ef7173f..bde1858 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 948cad4..424b3a7 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1475,3 +1475,1118 @@ explain (costs off) select * from like_op_noprune where a like '%BC';
 (5 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a3_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(19 rows)
+
+deallocate ab_q1;
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on ab_a1_b1 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a1_b2 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(13 rows)
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q2 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(24 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q3 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(24 rows)
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+drop table ab, lprt_a;
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(5 rows)
+
+deallocate q1;
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+   ->  Seq Scan on listp_2_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(5 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 08fc2db..73b4d10 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -253,3 +253,315 @@ create table like_op_noprune2 partition of like_op_noprune for values in ('BCD')
 explain (costs off) select * from like_op_noprune where a like '%BC';
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, hp, rp, coll_pruning_multi, like_op_noprune;
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- parallel append
+prepare ab_q2 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q3 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+execute ab_q3 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 3, 3);
+
+-- try some params whose values do not belong to any partition
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (33, 44, 55);
+
+-- test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q4 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+execute ab_q4 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+
+drop table ab, lprt_a;
+
+-- join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- no matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- test with columns defined in varying orders between each level
+
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- try with no matching partitions
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- both partitions allowed by IN clause, then both excluded again by <> clauses
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
-- 
1.9.5.msysgit.1

v14-0005-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchapplication/octet-stream; name=v14-0005-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchDownload
From 32dd4d7cd0306371a0b3116008aa783db5e472ca Mon Sep 17 00:00:00 2001
From: David Rowley <dgrowley@gmail.com>
Date: Fri, 2 Mar 2018 01:46:33 +1300
Subject: [PATCH v14 5/5] Allow MergeAppend's subnodes to be pruned during
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        |  52 ++++++++--
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 +++++++
 src/include/nodes/execnodes.h                 |   6 ++
 src/include/nodes/plannodes.h                 |   4 +
 src/test/regress/expected/partition_prune.out | 137 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  38 +++++++
 9 files changed, 269 insertions(+), 6 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef..eb55596 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -100,9 +101,6 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
 	 */
 
 	/*
@@ -153,6 +151,28 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 		PrepareSortSupportFromOrderingOp(node->sortOperators[i], sortKey);
 	}
 
+	if (node->part_prune_infos != NIL)
+	{
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		/*
+		 * When run-time partition pruning is enabled, setup the pruning data
+		 * structure.
+		 */
+		mergestate->partition_pruning =
+								ExecSetupPartitionPruning(&mergestate->ps,
+													node->part_prune_infos);
+	}
+	else
+	{
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
 	/*
 	 * initialize to show we have not run the subplans yet
 	 */
@@ -178,11 +198,18 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +322,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->prune_param_ids))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 65e02a2..80cc265 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -272,6 +272,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index a827cc6..1ada14d 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -433,6 +433,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index fa869a0..f403231 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1645,6 +1645,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1a788eb..eba525b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1125,6 +1125,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1210,8 +1212,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root, rel,
+											best_path->partitioned_rels,
+											best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 4141f34..70e6db7 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1046,6 +1046,10 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1058,6 +1062,8 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	PartitionPruning *partition_pruning;
+	Bitmapset   *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c94937edae..25e9574 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -274,6 +274,10 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 424b3a7..1875628 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2590,3 +2590,140 @@ select * from boolp where a = (select value from boolvalues where not value);
          Filter: (a = $0)
 (9 rows)
 
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p1.a
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(16 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p1.a
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(15 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 73b4d10..1f31204 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -565,3 +565,41 @@ select * from boolp where a = (select value from boolvalues where value);
 
 explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
1.9.5.msysgit.1

#92Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#91)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 03/01/2018 08:04 AM, David Rowley wrote:

I've also split the patch out a bit more into logical parts in the
hope it makes things easier to review.

A small typo in 0001:

+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one
bit in a

..."_one_"...

0004 fails "make check-world" due to

pg_restore: [archiver (db)] Error while PROCESSING TOC:
pg_restore: [archiver (db)] Error from TOC entry 670; 1259 49954 TABLE
boolp_f jpedersen
pg_restore: [archiver (db)] could not execute query: ERROR: syntax
error at or near "false"
LINE 24: ..." ATTACH PARTITION "public"."boolp_f" FOR VALUES IN (false);

Do you require https://commitfest.postgresql.org/17/1410/ as well ?

I'll look more at 0002-0005 over the coming days.

Thanks for working on this !

Best regards,
Jesper

#93David Rowley
david.rowley@2ndquadrant.com
In reply to: Jesper Pedersen (#92)
Re: [HACKERS] Runtime Partition Pruning

On 2 March 2018 at 07:17, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

A small typo in 0001:

+ * leftmost_ons_pos[x] gives the bit number (0-7) of the leftmost one bit
in a

..."_one_"...

Oops. I'll fix that.

0004 fails "make check-world" due to

pg_restore: [archiver (db)] Error while PROCESSING TOC:
pg_restore: [archiver (db)] Error from TOC entry 670; 1259 49954 TABLE
boolp_f jpedersen
pg_restore: [archiver (db)] could not execute query: ERROR: syntax error at
or near "false"
LINE 24: ..." ATTACH PARTITION "public"."boolp_f" FOR VALUES IN (false);

The tests seem to have stumbled on a pg_dump bug which causes it to
produce syntax that's not valid (currently)

I should be able to stop my patch failing the test by dropping that
table, which I should have been doing anyway.

Do you require https://commitfest.postgresql.org/17/1410/ as well ?

I'll look at that thread and see if there's any pg_dump being broken discussion.

I'll look more at 0002-0005 over the coming days.

Thanks for the review and in advance for the future review.

I'll delay releasing a new patch as there's some discussion over on
the faster partition pruning thread which affects this too [1]/messages/by-id/CA+Tgmoa4D1c4roj7L8cx8gkkeBWAZD=MTcXKxTwBnsLRHD3rig@mail.gmail.com

[1]: /messages/by-id/CA+Tgmoa4D1c4roj7L8cx8gkkeBWAZD=MTcXKxTwBnsLRHD3rig@mail.gmail.com

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#94Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#93)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 03/01/2018 08:29 PM, David Rowley wrote:

0004 fails "make check-world" due to

pg_restore: [archiver (db)] Error while PROCESSING TOC:
pg_restore: [archiver (db)] Error from TOC entry 670; 1259 49954 TABLE
boolp_f jpedersen
pg_restore: [archiver (db)] could not execute query: ERROR: syntax error at
or near "false"
LINE 24: ..." ATTACH PARTITION "public"."boolp_f" FOR VALUES IN (false);

The tests seem to have stumbled on a pg_dump bug which causes it to
produce syntax that's not valid (currently)

I should be able to stop my patch failing the test by dropping that
table, which I should have been doing anyway.

I've added that thread to the Open Items list.

Thanks for the review and in advance for the future review.

I'll delay releasing a new patch as there's some discussion over on
the faster partition pruning thread which affects this too [1]

[1] /messages/by-id/CA+Tgmoa4D1c4roj7L8cx8gkkeBWAZD=MTcXKxTwBnsLRHD3rig@mail.gmail.com

Sure, 0003-0005 depends on that thread. 0002 is refactoring so that one
is ready.

In 0004 should we add a HASH based test case,

-- test.sql --
-- verify pruning in hash partitions
create table hashp (a int primary key, b int) partition by hash (a);
create table hashp_0 partition of hashp for values with (modulus 2,
remainder 0);
create table hashp_1 partition of hashp for values with (modulus 2,
remainder 1);
insert into hashp values (0,0), (1,1), (2,2), (3,3);
prepare q1 (int) as select * from hashp where a = $1;
execute q1 (1);
execute q1 (1);
execute q1 (1);
execute q1 (1);
execute q1 (1);
explain (analyze, costs off, summary off, timing off) execute q1 (1);
explain (analyze, costs off, summary off, timing off) execute q1 (3);
deallocate q1;
drop table hashp;
-- test.sql --

Also, should 0004 consider the "Parallel Append" case, aka

-- parallel.sql --
CREATE TABLE t1 (
a integer NOT NULL,
b integer NOT NULL
) PARTITION BY HASH (b);

CREATE TABLE t1_p00 PARTITION OF t1 FOR VALUES WITH (MODULUS 4,
REMAINDER 0);
CREATE TABLE t1_p01 PARTITION OF t1 FOR VALUES WITH (MODULUS 4,
REMAINDER 1);
CREATE TABLE t1_p02 PARTITION OF t1 FOR VALUES WITH (MODULUS 4,
REMAINDER 2);
CREATE TABLE t1_p03 PARTITION OF t1 FOR VALUES WITH (MODULUS 4,
REMAINDER 3);
INSERT INTO t1 (SELECT i, i FROM generate_series(1, 1000000) AS i);
PREPARE q1 (int) AS SELECT * FROM t1 WHERE a = $1;
EXECUTE q1 (5432);
EXECUTE q1 (5432);
EXECUTE q1 (5432);
EXECUTE q1 (5432);
EXECUTE q1 (5432);
EXPLAIN (ANALYZE, COSTS OFF, SUMMARY OFF, TIMING OFF) EXECUTE q1 (5432);
DEALLOCATE q1;
DROP TABLE t1;
-- parallel.sql --

Best regards,
Jesper

#95David Rowley
david.rowley@2ndquadrant.com
In reply to: Jesper Pedersen (#94)
6 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 10 March 2018 at 07:50, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

On 03/01/2018 08:29 PM, David Rowley wrote:

I'll delay releasing a new patch as there's some discussion over on
the faster partition pruning thread which affects this too [1]

[1]
/messages/by-id/CA+Tgmoa4D1c4roj7L8cx8gkkeBWAZD=MTcXKxTwBnsLRHD3rig@mail.gmail.com

Sure, 0003-0005 depends on that thread. 0002 is refactoring so that one is
ready.

Okay, back to this again. The faster partition pruning patch has
undergone a large refactor which cause me to delay basing this patch
on top of it until that patch has stabilised again. It's now getting
close, so this seems like a good time to send a new verison of this.

In 0004 should we add a HASH based test case,

I don't think we really need to test each partition type in too much
detail in the run-time pruning patch. I'm more interested in testing
multiple partition levels there's important new code in this patch
which does need lots of additional testing for multi-level partitions.
I think adding a HASH partition case would be more aiming to test the
planner's pruning code, which this does not make any behavioural
changes to.

Also, should 0004 consider the "Parallel Append" case, aka

There is a parallel Append test case in there already, see the queries
below the "-- parallel append" comment.

I've attached a new version of the patch. I'm now at v18 after having
some versions of the patch that I didn't release which were based on
various versions of Amit's faster partition pruning patch.

The attached patchset is based on Amit's v45 faster partition pruning [1]/messages/by-id/fc73cef4-6879-26c3-6859-2f910640234a@lab.ntt.co.jp.

I've made a few changes since the v14 version. Since Amit's v45 patch
now creates the partition pruning details in a data structure that can
be copied from the planner over to the executor, it means this patch
is now able to do much of the setup work for run-time pruning in the
planner. Doing this now allows us to determine if run-time pruning is
even possible at plan time, rather than the executor attempting it and
sometimes wasting effort when it failed to find Params matching the
partition key.

Another change from the last version is that I've separated out the
handling of exec Params and external Params. The new patch now will
perform a pruning step at executor startup if some external params
match the partition key. This is very beneficial to a PREPAREd OLTP
type query against a partitioned table as it means we can skip
sub-plan initialisation for all non-matching partitions. Initialising
Append/MergeAppend/ModifyTable nodes with fewer subnodes than were
originally planned did require a small change in explain.c in some
code that was assuming the subnode arrays were the same length as the
subplan list. I also ended up adding a Int property to EXPLAIN to show
the number of subnodes that have been removed during execution.
Unfortunately, there is a small corner case with this in the case
where all subnodes are removed it leaves EXPLAIN with no subnodes to
search for outer side Vars in. I didn't really see any sane way to
handle this in EXPLAIN, so I think the best fix for this is what I've
done, and that's just to always initalise at least 1 subnode, even
when none match the external Params. Cost-wise, I don't think this is
such a big deal as the cost savings here are coming from saving on
initalising ten's or hundreds of subnodes, not 1.

To put the new patch to the test, I tried pgbench -S -M prepared -s
100 with and without having modified pgbench_accounts to separate into
10 RANGE partitions of equal size.

A non-partitioned table was getting 12503 TPS.
With partitioned tables, the old version of this patch was getting: 5470 TPS.
With partitioned tables, the attached version gets 11247 TPS.
For perspective, today's master with a partitioned table gets 4719 TPS.

So you can see it's a pretty good performance boost by skipping
initialisation of the 9 non-matching subplans. It's not hard to
imagine the gains getting more significant with a larger number of
partitions. Ideally, the performance of a partitioned table would be
faster than the non-partitioned case, but please remember this query
is a single row PK lookup SELECT, so is a very fast query in both
cases. Partitioning cases should improve more as the table grows and
indexes struggle to fit in RAM, or when many rows are being taken from
the partition and being aggregated.

Unfortunately, the story is not quite as good for the non -M prepared
version of the benchmark. This shows that planning time of partitioned
table queries could still use some improvements. Amit's v45 patch
certainly makes a dent in the planner slow performance here, but it's
still nothing like as fast as the non-partitioned case. More work is
required there in the future.

This patchset also contains a patch to improve the performance of
inheritance planning of UPDATE/DELETE queries. This patch also
implements run-time pruning for UPDATE/DELETE too. This also has a
significant performance improvement for planning of UPDATE/DELETE
operations on partitioned tables with a large number of partitions,
performance is as follows:

Values in transactions per second.

Partitions = 1
Unpatched: 7323.3
Patched: 6573.2 (-10.24%)

Partitions = 2
Unpatched: 6784.8
Patched: 6377.1 (-6.01%)

Partitions = 4
Unpatched: 5903.0
Patched: 6106.8 (3.45%)

Partitions = 8
Unpatched: 4582.0
Patched: 5579.9 (21.78%)

Partitions = 16
Unpatched: 3131.5
Patched: 4521.2 (44.38%)

Partitions = 32
Unpatched: 1779.8
Patched: 3387.8 (90.35%)

Partitions = 64
Unpatched: 821.9
Patched: 2245.4 (173.18%)

Partitions = 128
Unpatched: 322.2
Patched: 1319.6 (309.56%)

Partitions = 256
Unpatched: 84.3
Patched: 731.7 (768.27%)

Partitions = 512
Unpatched: 22.5
Patched: 382.8 (1597.74%)

Partitions = 1024
Unpatched: 5.5
Patched: 150.1 (2607.83%)

This shows a small regression in planner performance of 10% for a
table with 1 partition, but performance starts improving again by just
4 partitions. By the time we get to 1024 partitions the patched
planner is 26 times faster than unpatched. It's likely not a
no-brainer since some people may get a small performance regression,
but it certainly makes having larger numbers of partitions much more
usable than it was previously.

There are still a few bugs lingering in v45 of Amit's faster partition
pruning patch. One of which is causing a regression test to fail in
the attached patch set. I've also attached a patch which must be
applied after Amit's v45 patchset to fixup a couple of things missing.
Hopefully, this won't be needed once v46 is out.

To test this apply all patches in [1]/messages/by-id/fc73cef4-6879-26c3-6859-2f910640234a@lab.ntt.co.jp, then apply
faster_part_prune_v45_fixups.patch, then the attached 0001-0005
patches.

[1]: /messages/by-id/fc73cef4-6879-26c3-6859-2f910640234a@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

faster_part_prune_v45_fixups.patchapplication/octet-stream; name=faster_part_prune_v45_fixups.patchDownload
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 04a7e1aa62..22ed053532 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2141,6 +2141,7 @@ _copyPartitionPruneStepOp(const PartitionPruneStepOp *from)
 {
 	PartitionPruneStepOp *newnode = makeNode(PartitionPruneStepOp);
 
+	COPY_SCALAR_FIELD(step.step_id);
 	COPY_SCALAR_FIELD(opstrategy);
 	COPY_NODE_FIELD(exprs);
 	COPY_NODE_FIELD(cmpfns);
@@ -2156,7 +2157,7 @@ static PartitionPruneStepCombine *
 _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 {
 	PartitionPruneStepCombine *newnode = makeNode(PartitionPruneStepCombine);
-
+	COPY_SCALAR_FIELD(step.step_id);
 	COPY_SCALAR_FIELD(combineOp);
 	COPY_NODE_FIELD(source_stepids);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 9ce40ee3b3..3f9e2585c7 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1693,6 +1693,28 @@ _outFromExpr(StringInfo str, const FromExpr *node)
 	WRITE_NODE_FIELD(quals);
 }
 
+static void
+_outPartitionPruneStepOp(StringInfo str, const PartitionPruneStepOp *node)
+{
+	WRITE_NODE_TYPE("PARTITIONPRUNESTEPOP");
+
+	WRITE_INT_FIELD(step.step_id);
+	WRITE_INT_FIELD(opstrategy);
+	WRITE_NODE_FIELD(exprs);
+	WRITE_NODE_FIELD(cmpfns);
+	WRITE_BITMAPSET_FIELD(nullkeys);
+}
+
+static void
+_outPartitionPruneStepCombine(StringInfo str, const PartitionPruneStepCombine *node)
+{
+	WRITE_NODE_TYPE("PARTITIONPRUNESTEPCOMBINE");
+
+	WRITE_INT_FIELD(step.step_id);
+	WRITE_ENUM_FIELD(combineOp, PartitionPruneCombineOp);
+	WRITE_NODE_FIELD(source_stepids);
+}
+
 static void
 _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 {
@@ -3924,6 +3946,12 @@ outNode(StringInfo str, const void *obj)
 			case T_OnConflictExpr:
 				_outOnConflictExpr(str, obj);
 				break;
+			case T_PartitionPruneStepOp:
+				_outPartitionPruneStepOp(str, obj);
+				break;
+			case T_PartitionPruneStepCombine:
+				_outPartitionPruneStepCombine(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d02d4ec5b7..8348933151 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1328,6 +1328,32 @@ _readOnConflictExpr(void)
 	READ_DONE();
 }
 
+static PartitionPruneStepOp *
+_readPartitionPruneStepOp(void)
+{
+	READ_LOCALS(PartitionPruneStepOp);
+
+	READ_INT_FIELD(step.step_id);
+	READ_INT_FIELD(opstrategy);
+	READ_NODE_FIELD(exprs);
+	READ_NODE_FIELD(cmpfns);
+	READ_BITMAPSET_FIELD(nullkeys);
+
+	READ_DONE();
+}
+
+static PartitionPruneStepCombine *
+_readPartitionPruneStepCombine(void)
+{
+	READ_LOCALS(PartitionPruneStepCombine);
+
+	READ_INT_FIELD(step.step_id);
+	READ_ENUM_FIELD(combineOp, PartitionPruneCombineOp);
+	READ_NODE_FIELD(source_stepids);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2572,6 +2598,10 @@ parseNodeString(void)
 		return_value = _readFromExpr();
 	else if (MATCH("ONCONFLICTEXPR", 14))
 		return_value = _readOnConflictExpr();
+	else if (MATCH("PARTITIONPRUNESTEPOP", 20))
+		return_value = _readPartitionPruneStepOp();
+	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
+		return_value = _readPartitionPruneStepCombine();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
v18-0001-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v18-0001-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From 32c9deae63176e6705db8dcd4734ab678d4e8823 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sun, 1 Apr 2018 01:18:42 +1300
Subject: [PATCH v18 1/5] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/catalog/partition.c        |  23 ++
 src/backend/commands/explain.c         |  51 ++--
 src/backend/executor/execPartition.c   | 466 +++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c          |  20 ++
 src/backend/nodes/outfuncs.c           |  27 ++
 src/backend/nodes/readfuncs.c          |  19 ++
 src/backend/optimizer/util/partprune.c | 257 ++++++++++++++++++
 src/include/catalog/partition.h        |  13 +
 src/include/executor/execPartition.h   |  77 ++++++
 src/include/nodes/nodes.h              |   1 +
 src/include/nodes/primnodes.h          |  23 ++
 src/include/optimizer/partprune.h      |   4 +
 12 files changed, 963 insertions(+), 18 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index bcd282515a..3be53a9a83 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -2051,6 +2051,29 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			/*
+			 * 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;
+				bool isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+											context->planstate->ps_ExprContext,
+												   &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+
+			}
 		default:
 			break;
 	}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 8a58672a94..a3db51e660 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1808,28 +1808,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3140,18 +3145,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that
+	 * was specified in the plan means that some subnodes have been ignored
+	 * per instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 9a13188649..7c4f56c319 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,6 +39,12 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_extparams_recurse(
+									PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans);
+static void find_subplans_for_allparams_recurse(
+									PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans);
 
 
 /*
@@ -1151,3 +1157,463 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all. This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruning:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruning details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruning
+ *
+ * Setup the required data structure for calling ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These context can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionedRelPruning *partrelprunes;
+	PartitionPruning *partprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	partprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionedRelPruning *)
+					palloc(sizeof(PartitionedRelPruning) *
+						   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	partprune->partrelpruning = partrelprunes;
+	partprune->npartrelpruning = list_length(partitionpruneinfo);
+	partprune->extparams = NULL;
+	partprune->execparams = NULL;
+	partprune->allparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	partprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+													 "Partition Prune",
+													 ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionedRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc		partdesc;
+		Relation			rel;
+		PartitionKey		partkey;
+		int					partnatts;
+		int		j;
+
+		partrelprune->allpartindexes = bms_copy(pinfo->allpartindexes);
+		partrelprune->nparts = pinfo->nparts;
+		partrelprune->subnodeindex = palloc(sizeof(int) * pinfo->nparts);
+		partrelprune->subpartprune = palloc(sizeof(PartitionedRelPruning *) *
+										 pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(partrelprune->subnodeindex, pinfo->subnodeindex,
+			   sizeof(int) * pinfo->nparts);
+
+		for (j = 0; j < pinfo->nparts; j++)
+		{
+			int subpartidx = pinfo->subpartindex[j];
+
+			Assert(subpartidx < list_length(partitionpruneinfo));
+
+			if (subpartidx >= 0)
+				partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
+			else
+				partrelprune->subpartprune[j] = NULL;
+		}
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL;		/* empty for now */
+
+		partrelprune->prunesteps = pinfo->prunesteps;
+
+		partrelprune->extparams = bms_copy(pinfo->extparams);
+		partrelprune->allparams = bms_union(pinfo->extparams,
+											pinfo->execparams);
+
+		partprune->extparams = bms_add_members(partprune->extparams,
+											   pinfo->extparams);
+
+		partprune->execparams = bms_add_members(partprune->execparams,
+												pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the Param ids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	partprune->allparams = bms_union(partprune->extparams,
+									 partprune->execparams);
+
+	return partprune;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'partprune'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so once during their init
+ * plan.  The caller must only initialize the subnodes which are returned by
+ * this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'partprune' has any extparams.
+ *
+ * 'nsubplans' must be passed as the total number of unpruned subplans.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
+{
+	PartitionedRelPruning *partrelprune;
+	MemoryContext		oldcontext;
+	Bitmapset		   *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(partprune->extparams));
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	/* Determine which subplans match these external params */
+	find_subplans_for_extparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been done for external Params.
+	 */
+	bms_free(partprune->extparams);
+	partprune->extparams = NULL;
+
+	/*
+	 * If any subplans were pruned, we must re-sequence the subplan indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subplans which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubplans)
+	{
+		int	   *subplanidxmap;
+		int		i;
+		int		newidx;
+
+		/*
+		 * First we must build a map which allows us to map the old subplan
+		 * index into the new one.
+		 */
+		subplanidxmap = (int *) palloc(sizeof(int) * nsubplans);
+		newidx = 0;
+		for (i = 0; i < nsubplans; i++)
+		{
+			if (bms_is_member(i, result))
+				subplanidxmap[i] = newidx++;
+			else
+				subplanidxmap[i] = -1;		/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnodeindex
+		 * so that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < partprune->npartrelpruning; i++)
+		{
+			PartitionedRelPruning *partrelprune;
+			int j;
+
+			partrelprune = &partprune->partrelpruning[i];
+
+			/*
+			 * We also need to reset the allpartindexes field so that it
+			 * only contains partition indexes that we actually still have
+			 * subnodeindexes for.  It seems easier to build a fresh one,
+			 * rather than trying to update the existing one.
+			 */
+			bms_free(partrelprune->allpartindexes);
+			partrelprune->allpartindexes = NULL;
+
+			for (j = 0; j < partrelprune->nparts; j++)
+			{
+				int oldidx = partrelprune->subnodeindex[j];
+
+				/*
+				 * If this partition existed as a subplan then change the old
+				 * subplan index to the new subplan index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subplan list due to some subplans
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					partrelprune->subnodeindex[j] = subplanidxmap[oldidx];
+
+					if (subplanidxmap[oldidx] >= 0)
+						partrelprune->allpartindexes =
+								bms_add_member(partrelprune->allpartindexes,
+											   j);
+				}
+			}
+		}
+
+		pfree(subplanidxmap);
+	}
+
+
+	return result;
+}
+
+/*
+ * find_subplans_for_extparams_recurse
+ *		Recursive worker function for ExecFindInitialMatchingSubPlans.
+ */
+static void
+find_subplans_for_extparams_recurse(PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset	   *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * We only need to determine the matching partitions if there are any
+	 * ext params matching the partition key at this level.  If there are no
+	 * such 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.
+	 */
+	if (!bms_is_empty(partrelprune->extparams))
+	{
+		context->safeparams = partrelprune->extparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_extparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'partprune' for the current Param values.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *partprune)
+{
+	PartitionedRelPruning *partrelprune;
+	MemoryContext		oldcontext;
+	Bitmapset *result = NULL;
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	find_subplans_for_allparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_allparams_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans.
+ */
+static void
+find_subplans_for_allparams_recurse(PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset	   *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(partrelprune->allparams))
+	{
+		context->safeparams = partrelprune->allparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_allparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 22ed053532..d63fcf782e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2164,6 +2164,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(prunesteps);
+	COPY_BITMAPSET_FIELD(allpartindexes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5068,6 +5085,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 3f9e2585c7..bd62f81cea 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1730,6 +1730,30 @@ _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(prunesteps);
+	WRITE_BITMAPSET_FIELD(allpartindexes);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnodeindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnodeindex[i]);
+
+	appendStringInfoString(str, " :subpartindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpartindex[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3952,6 +3976,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 8348933151..5200f3cd8a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1354,6 +1354,23 @@ _readPartitionPruneStepCombine(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(prunesteps);
+	READ_BITMAPSET_FIELD(allpartindexes);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnodeindex, local_node->nparts);
+	READ_INT_ARRAY(subpartindex, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2602,6 +2619,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index e5e6d7530b..45463d7ed8 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -32,6 +32,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -85,6 +86,7 @@ typedef struct GeneratePruningStepsContext
 	List   *steps;
 }			GeneratePruningStepsContext;
 
+static bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static List *generate_partition_pruning_steps_internal(RelOptInfo *rel,
 									  GeneratePruningStepsContext *context,
 									  List *clauses,
@@ -169,6 +171,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 		context.nparts = rel->nparts;
 		context.boundinfo = rel->boundinfo;
 
+		/* Not valid when being called from the planner */
+		context.planstate = NULL;
+		context.safeparams = NULL;
+
 		partindexes = get_matching_partitions(&context, pruning_steps);
 
 		/* Add selected partitions' RT indexes to result. */
@@ -232,8 +238,259 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 	return context.steps;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Return a List of PartitionPruneInfos, one for each 'partitioned_rel',
+ *		or NIL if no Params were found matching the partition key, in which
+ *		case run-time partition pruning is useless.
+ *
+ * Here we index the subpaths by partition index so that we're able to
+ * translate the output of get_matching_partitions into subpath indexes to
+ * possibly allow for further partition pruning to be performed during
+ * execution.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal)
+{
+	RangeTblEntry	   *rte;
+	RelOptInfo		   *parentpart;
+	ListCell		   *lc;
+	List			   *pinfolist = NIL;
+	int				   *allsubnodeindex;
+	int				   *allsubpartindex;
+	int					i;
+	bool				gotparam = false;
+
+	/*
+	 * Allocate two arrays, one to allow quick lookups of the 'subpaths' index
+	 * of a relation by relid and another to lookup the 'partitioned_rel'
+	 * index by relid.
+	 */
+	allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
+	allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);
+
+	/* Initialize to -1 to indicate the rel was not found */
+	for (i = 0; i < root->simple_rel_array_size; i++)
+	{
+		allsubnodeindex[i] = -1;
+		allsubpartindex[i] = -1;
+	}
+
+	/*
+	 * Now loop over each subpath and fill in the index of the subpath for the
+	 * subpath's relid.
+	 */
+	if (resultRelations != NIL)
+	{
+		i = 0;
+		foreach(lc, resultRelations)
+		{
+			int resultrel = lfirst_int(lc);
+			Assert(resultrel < root->simple_rel_array_size);
+			allsubnodeindex[resultrel] = i;
+			i++;
+		}
+	}
+	else
+	{
+		i = 0;
+		foreach(lc, subpaths)
+		{
+			Path *path = (Path *) lfirst(lc);
+			RelOptInfo *pathrel = path->parent;
+
+			Assert(IS_SIMPLE_REL(pathrel));
+			Assert(pathrel->relid < root->simple_rel_array_size);
+
+			allsubnodeindex[pathrel->relid] = i;
+			i++;
+		}
+	}
+
+	/* Likewise for the partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index	rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		allsubpartindex[rti] = i;
+		i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		int			nparts = subpart->nparts;
+		int		   *subnodeindex;
+		int		   *subpartindex;
+		List	   *partprunequal;
+		bool		constfalse;
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (i == 0)
+		{
+			parentpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+						adjust_appendrel_attrs_multilevel(root,
+														  (Node *) prunequal,
+														  subpart->relids,
+														  parentpart->relids);
+		}
+
+		pinfo->prunesteps = generate_partition_pruning_steps(subpart,
+															 partprunequal,
+															 &constfalse);
+
+		if (constfalse)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered.  We'd
+			 * better do something sane here anyway, so let's just disable
+			 * run-time pruning.
+			 */
+			return NIL;
+		}
+
+		pinfo->allpartindexes = NULL;
+		pinfo->nparts = nparts;
+		pinfo->subnodeindex = subnodeindex = palloc(nparts * sizeof(int));
+		pinfo->subpartindex = subpartindex = palloc(nparts * sizeof(int));
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+
+		/*
+		 * 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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pinfo->prunesteps);
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present
+		 * in the subpaths List will be set to -1, and any subpartitioned
+		 * table which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = allsubnodeindex[partrel->relid];
+			int			subpartidx = allsubpartindex[partrel->relid];
+
+			subnodeindex[i] = subnodeidx;
+			subpartindex[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				pinfo->allpartindexes = bms_add_member(pinfo->allpartindexes,
+													   i);
+		}
+
+		pinfolist = lappend(pinfolist, pinfo);
+		i++;
+	}
+
+	pfree(allsubnodeindex);
+	pfree(allsubpartindex);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
+
 /* Module-local functions */
 
+/*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							(int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
 /*
  * generate_partition_pruning_steps_internal
  *		Processes 'clauses' to generate partition pruning steps.
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8981901272..558a996ac3 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -62,6 +63,18 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow
+	 * resolution of Param values.
+	 */
+	PlanState	*planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until after init plan.
+	 */
+	Bitmapset *safeparams;
 } PartitionPruneContext;
 
 extern void RelationBuildPartitionDesc(Relation relation);
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 03a599ad57..63e287ea81 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "optimizer/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,77 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionedRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows partprune.c's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'prunesteps' will be found within.
+ *
+ * nparts						The number of partitions which belong to this
+ *								partitioned relation. Also defines the size of
+ *								the 'subnodeindex' and 'subpartprune' arrays.
+ * subnodeindex					An array of nparts containing the subnode
+ *								index which matches this partition index, or
+ *								-1 if there is no match.
+ * subpartprune					An array of nparts containing the
+ *								PartitionedRelPruning details this partition
+ *								index for sub-partitioned tables.
+ * allpartindexes				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ *								belong to this partition.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * prunesteps					Contains list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ *-----------------------
+ */
+typedef struct PartitionedRelPruning
+{
+	int			nparts;
+	int		   *subnodeindex;
+	struct PartitionedRelPruning **subpartprune;
+	Bitmapset  *allpartindexes;
+	PartitionPruneContext context;
+	List	   *prunesteps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionedRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionedRelPruning
+ * structs and also stores all Param IDs which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partrelpruning		Array of PartitionedRelPruning for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * npartrelpruning		Number of items in partrelpruning array.
+ * prune_context		A memory context which can be used to call the query
+ *						planner's partition prune functions.
+ * extparams			All PARAM_EXTERN Param IDs which were found to match a
+ *						partition key in each of the contained
+ *						PartitionedRelPruning structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of extparams and execparams, saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionedRelPruning *partrelpruning;
+	int npartrelpruning;
+	MemoryContext prune_context;
+	Bitmapset *extparams;
+	Bitmapset *execparams;
+	Bitmapset *allparams;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -126,5 +198,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+								int nsubplans);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index e6b5770c74..adb159a6da 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -195,6 +195,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index a71d729e72..c9dad76755 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1580,4 +1580,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;		/* Oid of partition rel */
+	List	   *prunesteps;	/* List of PartitionPruneStep */
+	Bitmapset  *allpartindexes;	/* All part index we have subnodes for at this
+								 * level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+	int		   *subpartindex;	/* subpart index indexed by partition id */
+	Bitmapset  *extparams;		/* All external ParamIDs seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec ParamIDs seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 1f2fe297a3..b7352d150c 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -20,4 +20,8 @@ extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
-- 
2.16.2.windows.1

v18-0002-Add-bms_prev_member-function.patchapplication/octet-stream; name=v18-0002-Add-bms_prev_member-function.patchDownload
From b00562c1a115c3646b698be18ccafc76fa44d13d Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sun, 1 Apr 2018 01:19:11 +1300
Subject: [PATCH v18 2/5] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1088,6 +1110,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 	return -2;
 }
 
+/*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
 /*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.16.2.windows.1

v18-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchapplication/octet-stream; name=v18-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchDownload
From a5eb46a0940f23927a1dac2a05201d47ab429609 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sun, 1 Apr 2018 01:19:41 +1300
Subject: [PATCH v18 3/5] Allow Append subnodes to be pruned during execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  258 ++++--
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   15 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1800 insertions(+), 88 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..5286ada835 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +101,9 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i, j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -112,55 +115,117 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+	appendstate->as_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&appendstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set a noop flag so that we never actually
+			 * bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+							bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+												 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
-	 */
-	i = 0;
+		* call ExecInitNode on each of the valid plans to be executed and save
+		* the results into the appendplanstates array.
+		*/
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
-
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
+
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +316,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +348,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +438,40 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/* Handle the case for when all subplans were pruned */
+	if (node->as_noopscan)
+		return false;
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -411,10 +500,33 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
 	else
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +579,25 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
+
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +663,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d63fcf782e..b67c5b86ab 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -245,6 +245,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index 52de893e89..a10014f755 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3748,32 +3748,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3823,15 +3823,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index bd62f81cea..3a88ccbea9 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -402,6 +402,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 5200f3cd8a..1632c13ce9 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1651,6 +1651,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index c36a254ed6..25cfa5e349 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1607,7 +1607,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1649,8 +1649,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1698,7 +1698,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1761,7 +1761,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2027,7 +2027,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 477b11f11d..f498952118 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1037,6 +1038,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1074,6 +1077,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels, NIL,
+											best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1082,7 +1116,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5375,7 +5410,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5387,7 +5423,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 3a0ac754d8..556efd8ac6 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3906,7 +3906,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6818,8 +6819,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 058fb24927..4eec62adb2 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 22133fcf12..d4666a31bd 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3567,7 +3585,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6070a42b6f..3888674389 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1051,8 +1051,15 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
+ *		noopscan			true if partition pruning proved that none of the
+ *							appendplans can contain a record to satisfy this
+ *							query.
  * ----------------
  */
 
@@ -1060,6 +1067,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruning;
 
 struct AppendState
 {
@@ -1069,7 +1077,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset   *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
+	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c922216b7d..e33799d919 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -252,6 +252,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 381bc30813..e99ae36bef 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 0be31cce7e..ef8537f3ac 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1517,3 +1517,1138 @@ explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 (9 rows)
 
 drop table hp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 8377671cde..dc1ae6d975 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -276,3 +276,347 @@ explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and
 explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 
 drop table hp;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.16.2.windows.1

v18-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchapplication/octet-stream; name=v18-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchDownload
From 73aa4192fec1e8388873941d9072a0ea5c80ee96 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sun, 1 Apr 2018 01:20:08 +1300
Subject: [PATCH v18 4/5] Allow MergeAppend's subnodes to be pruned during
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   4 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 351 insertions(+), 22 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..d9cf911f4f 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,9 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i, j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -78,19 +80,89 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mergestate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+							bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +173,23 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
-
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +247,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +374,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b67c5b86ab..9f2fd52a0b 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -273,6 +273,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 3a88ccbea9..cace730efa 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -434,6 +434,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 1632c13ce9..31fed08627 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1673,6 +1673,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index f498952118..090b22224c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1140,6 +1140,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1225,8 +1227,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels, NIL,
+											best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 3888674389..681779d42f 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1092,6 +1092,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1104,6 +1110,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset   *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index e33799d919..c90c763a86 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -276,6 +276,10 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index ef8537f3ac..bb54924578 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2652,3 +2652,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index dc1ae6d975..e87cf52536 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -620,3 +620,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.16.2.windows.1

v18-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchapplication/octet-stream; name=v18-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchDownload
From 2a301f59561ac9f5d836122ae26ec4609033392e Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sun, 1 Apr 2018 01:20:47 +1300
Subject: [PATCH v18 5/5] Improve planning speed of partitioned table
 UPDATE/DELETEs

By making a call to grouping_planner for the complete parse of the query we
can make use of the faster partition pruning code used there.  This will
identify all partitions which could be pruned as IS_DUMMY_RELs, of which we
can skip performing each individual grouping_planner call inside
inheritance_planner.  This can improve planner performance significantly when
there are many partitions. There may be a slight slowdown when no partitions
could be pruned or when there are very few (1 or 2) partitions.  However it
seems better to optimize the case when partitions are pruned, rather than the
case where they're not, as those queries are less likely to be fast to
execute. The case for partitioned tables with just 1 or 2 leaf partitions does
not seem worth worrying about too much. The measured regression on 1 partition
was just 10% of overall planning time.

This commit also implements run-time partition pruning for UPDATE/DELETE.
---
 src/backend/commands/explain.c          |   4 +-
 src/backend/executor/execPartition.c    |  14 +--
 src/backend/executor/nodeModifyTable.c  | 159 ++++++++++++++++++++++++--------
 src/backend/nodes/copyfuncs.c           |   1 +
 src/backend/nodes/outfuncs.c            |   1 +
 src/backend/nodes/readfuncs.c           |   1 +
 src/backend/optimizer/plan/createplan.c |  32 ++++++-
 src/backend/optimizer/plan/planner.c    |  59 ++++++++++++
 src/include/nodes/execnodes.h           |  11 ++-
 src/include/nodes/plannodes.h           |   2 +
 10 files changed, 233 insertions(+), 51 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index a3db51e660..865ba928d3 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -3021,14 +3021,14 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 	/* Should we explicitly label target relations? */
 	labeltargets = (mtstate->mt_nplans > 1 ||
 					(mtstate->mt_nplans == 1 &&
-					 mtstate->resultRelInfo->ri_RangeTableIndex != node->nominalRelation));
+					 mtstate->resultRelInfos[0]->ri_RangeTableIndex != node->nominalRelation));
 
 	if (labeltargets)
 		ExplainOpenGroup("Target Tables", "Target Tables", false, es);
 
 	for (j = 0; j < mtstate->mt_nplans; j++)
 	{
-		ResultRelInfo *resultRelInfo = mtstate->resultRelInfo + j;
+		ResultRelInfo *resultRelInfo = mtstate->resultRelInfos[j];
 		FdwRoutine *fdwroutine = resultRelInfo->ri_FdwRoutine;
 
 		if (labeltargets)
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 7c4f56c319..fb46a458b0 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -96,7 +96,7 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
 	/* Set up details specific to the type of tuple routing we are doing. */
 	if (node && node->operation == CMD_UPDATE)
 	{
-		update_rri = mtstate->resultRelInfo;
+		update_rri = mtstate->resultRelInfos[0];
 		num_update_rri = list_length(node->plans);
 		proute->subplan_partition_offsets =
 			palloc(num_update_rri * sizeof(int));
@@ -386,8 +386,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * In the case of INSERT on a partitioned table, there is only one
@@ -442,8 +442,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		TupleTableSlot *slot;
 		ExprContext *econtext;
 		List	   *returningList;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/* See the comment above for WCO lists. */
 		Assert((node->operation == CMD_INSERT &&
@@ -498,8 +498,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	if (node && node->onConflictAction != ONCONFLICT_NONE)
 	{
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		ExprContext *econtext = mtstate->ps.ps_ExprContext;
 		ListCell   *lc;
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 1b09868ff8..3afa9d07c8 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -1116,14 +1116,10 @@ lreplace:;
 				saved_tcs_map = mtstate->mt_transition_capture->tcs_map;
 
 			/*
-			 * resultRelInfo is one of the per-subplan resultRelInfos.  So we
-			 * should convert the tuple into root's tuple descriptor, since
-			 * ExecInsert() starts the search from root.  The tuple conversion
-			 * map list is in the order of mtstate->resultRelInfo[], so to
-			 * retrieve the one for this resultRel, we need to know the
-			 * position of the resultRel in mtstate->resultRelInfo[].
+			 * Convert the tuple into root's tuple descriptor, since
+			 * ExecInsert() starts the search from root.
 			 */
-			map_index = resultRelInfo - mtstate->resultRelInfo;
+			map_index = mtstate->mt_whichplan;
 			Assert(map_index >= 0 && map_index < mtstate->mt_nplans);
 			tupconv_map = tupconv_map_for_subplan(mtstate, map_index);
 			tuple = ConvertPartitionTupleSlot(tupconv_map,
@@ -1503,12 +1499,12 @@ static void
 fireBSTriggers(ModifyTableState *node)
 {
 	ModifyTable *plan = (ModifyTable *) node->ps.plan;
-	ResultRelInfo *resultRelInfo = node->resultRelInfo;
+	ResultRelInfo *resultRelInfo = node->resultRelInfos[0];
 
 	/*
 	 * If the node modifies a partitioned table, we must fire its triggers.
-	 * Note that in that case, node->resultRelInfo points to the first leaf
-	 * partition, not the root table.
+	 * Note that in that case, node->resultRelInfos[0] points to the first
+	 * leaf partition, not the root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		resultRelInfo = node->rootResultRelInfo;
@@ -1546,13 +1542,14 @@ static ResultRelInfo *
 getTargetResultRelInfo(ModifyTableState *node)
 {
 	/*
-	 * Note that if the node modifies a partitioned table, node->resultRelInfo
-	 * points to the first leaf partition, not the root table.
+	 * Note that if the node modifies a partitioned table,
+	 * node->resultRelInfos[0] points to the first leaf partition, not the
+	 * root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		return node->rootResultRelInfo;
 	else
-		return node->resultRelInfo;
+		return node->resultRelInfos[0];
 }
 
 /*
@@ -1762,7 +1759,7 @@ static void
 ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 {
 	ResultRelInfo *targetRelInfo = getTargetResultRelInfo(mtstate);
-	ResultRelInfo *resultRelInfos = mtstate->resultRelInfo;
+	ResultRelInfo **resultRelInfos = mtstate->resultRelInfos;
 	TupleDesc	outdesc;
 	int			numResultRelInfos = mtstate->mt_nplans;
 	int			i;
@@ -1793,7 +1790,7 @@ ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 	for (i = 0; i < numResultRelInfos; ++i)
 	{
 		mtstate->mt_per_subplan_tupconv_maps[i] =
-			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i].ri_RelationDesc),
+			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i]->ri_RelationDesc),
 								   outdesc,
 								   gettext_noop("could not convert row type"));
 	}
@@ -1932,7 +1929,7 @@ ExecModifyTable(PlanState *pstate)
 	}
 
 	/* Preload local variables */
-	resultRelInfo = node->resultRelInfo + node->mt_whichplan;
+	resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 	subplanstate = node->mt_plans[node->mt_whichplan];
 	junkfilter = resultRelInfo->ri_junkFilter;
 
@@ -1969,7 +1966,7 @@ ExecModifyTable(PlanState *pstate)
 			node->mt_whichplan++;
 			if (node->mt_whichplan < node->mt_nplans)
 			{
-				resultRelInfo++;
+				resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 				subplanstate = node->mt_plans[node->mt_whichplan];
 				junkfilter = resultRelInfo->ri_junkFilter;
 				estate->es_result_relation_info = resultRelInfo;
@@ -2153,9 +2150,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	ResultRelInfo *resultRelInfo;
 	Plan	   *subplan;
 	ListCell   *l;
-	int			i;
+	int			i, j;
 	Relation	rel;
 	bool		update_tuple_routing_needed = node->partColsUpdated;
+	Bitmapset  *validsubplans;
 
 	/* check for unsupported flags */
 	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -2172,8 +2170,70 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	mtstate->canSetTag = node->canSetTag;
 	mtstate->mt_done = false;
 
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mtstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mtstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away ModifyTable plans.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->plans));
+
+			/*
+			 * If no plans match the given parameters then we must handle this
+			 * case in a special way.  The problem here is that code in
+			 * explain.c requires a ModifyTable to have at least one plan in
+			 * order for it to properly determine the Vars in that plan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set the mt_done flag so that we never
+			 * actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mtstate->mt_done = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->plans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		mtstate->partition_pruning = partprune;
+	}
+	else
+	{
+		nplans = list_length(node->plans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all plans as valid, they must also all be initialized.
+		 */
+		validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		mtstate->partition_pruning = NULL;
+	}
+
+
 	mtstate->mt_plans = (PlanState **) palloc0(sizeof(PlanState *) * nplans);
-	mtstate->resultRelInfo = estate->es_result_relations + node->resultRelIndex;
+	mtstate->resultRelInfos = (ResultRelInfo **)
+									palloc(sizeof(ResultRelInfo *) * nplans);
 
 	/* If modifying a partitioned table, initialize the root table info */
 	if (node->rootResultRelIndex >= 0)
@@ -2197,12 +2257,20 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	 */
 	saved_resultRelInfo = estate->es_result_relation_info;
 
-	resultRelInfo = mtstate->resultRelInfo;
-	i = 0;
+	j = i = 0;
 	foreach(l, node->plans)
 	{
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
 		subplan = (Plan *) lfirst(l);
 
+		resultRelInfo = estate->es_result_relations + node->resultRelIndex + i;
+		mtstate->resultRelInfos[j] = resultRelInfo;
+
 		/* Initialize the usesFdwDirectModify flag */
 		resultRelInfo->ri_usesFdwDirectModify = bms_is_member(i,
 															  node->fdwDirectModifyPlans);
@@ -2239,7 +2307,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		/* Now init the plan for this result rel */
 		estate->es_result_relation_info = resultRelInfo;
-		mtstate->mt_plans[i] = ExecInitNode(subplan, estate, eflags);
+		mtstate->mt_plans[j] = ExecInitNode(subplan, estate, eflags);
 
 		/* Also let FDWs init themselves for foreign-table result rels */
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
@@ -2255,8 +2323,8 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 															 eflags);
 		}
 
-		resultRelInfo++;
 		i++;
+		j++;
 	}
 
 	estate->es_result_relation_info = saved_resultRelInfo;
@@ -2301,26 +2369,34 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	/*
 	 * Initialize any WITH CHECK OPTION constraints if needed.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
-	i = 0;
+	j = i = 0;
 	foreach(l, node->withCheckOptionLists)
 	{
-		List	   *wcoList = (List *) lfirst(l);
+		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
 
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
+		wcoList = (List *) lfirst(l);
+
 		foreach(ll, wcoList)
 		{
 			WithCheckOption *wco = (WithCheckOption *) lfirst(ll);
 			ExprState  *wcoExpr = ExecInitQual((List *) wco->qual,
-											   mtstate->mt_plans[i]);
+											   mtstate->mt_plans[j]);
 
 			wcoExprs = lappend(wcoExprs, wcoExpr);
 		}
-
+		resultRelInfo = mtstate->resultRelInfos[j];
 		resultRelInfo->ri_WithCheckOptions = wcoList;
 		resultRelInfo->ri_WithCheckOptionExprs = wcoExprs;
-		resultRelInfo++;
+
+		j++;
 		i++;
 	}
 
@@ -2350,15 +2426,25 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		/*
 		 * Build a projection for each result rel.
 		 */
-		resultRelInfo = mtstate->resultRelInfo;
+		j = i = 0;
 		foreach(l, node->returningLists)
 		{
-			List	   *rlist = (List *) lfirst(l);
+			List	   *rlist;
+
+			if (!bms_is_member(i, validsubplans))
+			{
+				i++;
+				continue;
+			}
+
+			rlist  = (List *) lfirst(l);
 
+			resultRelInfo = mtstate->resultRelInfos[j];
 			resultRelInfo->ri_projectReturning =
 				ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps,
 										resultRelInfo->ri_RelationDesc->rd_att);
-			resultRelInfo++;
+			j++;
+			i++;
 		}
 	}
 	else
@@ -2374,7 +2460,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	}
 
 	/* Set the list of arbiter indexes if needed for ON CONFLICT */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (node->onConflictAction != ONCONFLICT_NONE)
 		resultRelInfo->ri_onConflictArbiterIndexes = node->arbiterIndexes;
 
@@ -2528,11 +2614,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		if (junk_filter_needed)
 		{
-			resultRelInfo = mtstate->resultRelInfo;
 			for (i = 0; i < nplans; i++)
 			{
 				JunkFilter *j;
 
+				resultRelInfo = mtstate->resultRelInfos[i];
 				subplan = mtstate->mt_plans[i]->plan;
 				if (operation == CMD_INSERT || operation == CMD_UPDATE)
 					ExecCheckPlanOutput(resultRelInfo->ri_RelationDesc,
@@ -2573,13 +2659,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 				}
 
 				resultRelInfo->ri_junkFilter = j;
-				resultRelInfo++;
 			}
 		}
 		else
 		{
 			if (operation == CMD_INSERT)
-				ExecCheckPlanOutput(mtstate->resultRelInfo->ri_RelationDesc,
+				ExecCheckPlanOutput(mtstate->resultRelInfos[0]->ri_RelationDesc,
 									subplan->targetlist);
 		}
 	}
@@ -2626,7 +2711,7 @@ ExecEndModifyTable(ModifyTableState *node)
 	 */
 	for (i = 0; i < node->mt_nplans; i++)
 	{
-		ResultRelInfo *resultRelInfo = node->resultRelInfo + i;
+		ResultRelInfo *resultRelInfo = node->resultRelInfos[i];
 
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
 			resultRelInfo->ri_FdwRoutine != NULL &&
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9f2fd52a0b..4b40b0f30f 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -222,6 +222,7 @@ _copyModifyTable(const ModifyTable *from)
 	COPY_NODE_FIELD(onConflictWhere);
 	COPY_SCALAR_FIELD(exclRelRTI);
 	COPY_NODE_FIELD(exclRelTlist);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index cace730efa..2348fc5369 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -390,6 +390,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 	WRITE_NODE_FIELD(onConflictWhere);
 	WRITE_UINT_FIELD(exclRelRTI);
 	WRITE_NODE_FIELD(exclRelTlist);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 31fed08627..12da032528 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1634,6 +1634,7 @@ _readModifyTable(void)
 	READ_NODE_FIELD(onConflictWhere);
 	READ_UINT_FIELD(exclRelRTI);
 	READ_NODE_FIELD(exclRelTlist);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 090b22224c..4437e28902 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -291,7 +291,8 @@ static ModifyTable *make_modifytable(PlannerInfo *root,
 				 bool partColsUpdated,
 				 List *resultRelations, List *subplans,
 				 List *withCheckOptionLists, List *returningLists,
-				 List *rowMarks, OnConflictExpr *onconflict, int epqParam);
+				 List *rowMarks, OnConflictExpr *onconflict, int epqParam,
+				 List *partpruneinfos);
 static GatherMerge *create_gather_merge_plan(PlannerInfo *root,
 						 GatherMergePath *best_path);
 
@@ -2480,6 +2481,7 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 	List	   *subplans = NIL;
 	ListCell   *subpaths,
 			   *subroots;
+	List	   *partpruneinfos = NIL;
 
 	/* Build the plan for each input path */
 	forboth(subpaths, best_path->subpaths,
@@ -2508,6 +2510,27 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (best_path->partitioned_rels != NIL)
+	{
+		int			partrelid = linitial_int(best_path->partitioned_rels);
+		RelOptInfo *rel = root->simple_rel_array[partrelid];
+		List	   *prunequal = NIL;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels,
+											best_path->resultRelations,
+											best_path->subpaths, prunequal);
+	}
+
 	plan = make_modifytable(root,
 							best_path->operation,
 							best_path->canSetTag,
@@ -2520,7 +2543,8 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 							best_path->returningLists,
 							best_path->rowMarks,
 							best_path->onconflict,
-							best_path->epqParam);
+							best_path->epqParam,
+							partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, &best_path->path);
 
@@ -6589,7 +6613,8 @@ make_modifytable(PlannerInfo *root,
 				 bool partColsUpdated,
 				 List *resultRelations, List *subplans,
 				 List *withCheckOptionLists, List *returningLists,
-				 List *rowMarks, OnConflictExpr *onconflict, int epqParam)
+				 List *rowMarks, OnConflictExpr *onconflict, int epqParam,
+				 List *partpruneinfos)
 {
 	ModifyTable *node = makeNode(ModifyTable);
 	List	   *fdw_private_list;
@@ -6648,6 +6673,7 @@ make_modifytable(PlannerInfo *root,
 	node->returningLists = returningLists;
 	node->rowMarks = rowMarks;
 	node->epqParam = epqParam;
+	node->part_prune_infos = partpruneinfos;
 
 	/*
 	 * For each result relation that is a foreign table, allow the FDW to
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 556efd8ac6..41e0dc4e70 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1179,6 +1179,7 @@ inheritance_planner(PlannerInfo *root)
 	Query	   *parent_parse;
 	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
 	PlannerInfo **parent_roots = NULL;
+	PlannerInfo *partition_root = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1256,6 +1257,32 @@ inheritance_planner(PlannerInfo *root)
 		 * the ModifyTable node, if one is needed at all.
 		 */
 		partitioned_relids = bms_make_singleton(top_parentRTindex);
+
+		/*
+		 * For partitioned tables, since we're able to determine the minimum
+		 * set of partitions required much more easily than what we can do
+		 * with an inheritance hierarchy, we invoke the grouping_planner on
+		 * the entire given query in order to determine the minimum set of
+		 * partitions which will be required below.  This may mean that we
+		 * invoke the grouping planner far fewer times, as otherwise we'd
+		 * have to invoke it once for each partition.
+		 */
+
+		/*
+		 * Since the planner tends to scribble on the parse, we must make a
+		 * copy of it.  We also must make copies of the PlannerInfo and
+		 * PlannerGlobal since these will also be modified from the call to
+		 * grouping_planner.
+		 */
+		partition_root = makeNode(PlannerInfo);
+		partition_root->glob = makeNode(PlannerGlobal);
+
+		memcpy(partition_root, root, sizeof(PlannerInfo));
+		memcpy(partition_root->glob, root->glob, sizeof(PlannerGlobal));
+
+		partition_root->parse = copyObject(partition_root->parse);
+
+		grouping_planner(partition_root, true, 0.0 /* retrieve all tuples */ );
 	}
 
 	/*
@@ -1286,6 +1313,21 @@ inheritance_planner(PlannerInfo *root)
 		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
+		/*
+		 * If the target rel is a partitioned table then skip any child
+		 * partitions which were found to be dummies by the grouping_planner
+		 * call performed above.
+		 */
+		if (partition_root)
+		{
+			RelOptInfo *rel;
+
+			rel = find_base_rel(partition_root, appinfo->child_relid);
+
+			if (IS_DUMMY_REL(rel))
+				continue;
+		}
+
 		/*
 		 * expand_inherited_rtentry() always processes a parent before any of
 		 * that parent's children, so the parent_root for this relation should
@@ -1610,6 +1652,23 @@ inheritance_planner(PlannerInfo *root)
 		Assert(list_length(partitioned_rels) >= 1);
 	}
 
+	/*
+	 * The individual grouping_planner calls per partition above performed
+	 * no planning on the actual partitioned tables, however, in order to
+	 * allow partition pruning at run-time we must know the baserestrictinfo
+	 * of each partition.  We simply replace the RelOptInfos from the initial
+	 * full plan which was generated and replace the non-complete RelOptInfos
+	 * which are stored in root.
+	 */
+	if (partition_root)
+	{
+		int		i;
+
+		i = -1;
+		while ((i = bms_next_member(partitioned_relids, i)) >= 0)
+			root->simple_rel_array[i] = partition_root->simple_rel_array[i];
+	}
+
 	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
 	add_path(final_rel, (Path *)
 			 create_modifytable_path(root, final_rel,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 681779d42f..65de9ef6f8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1012,6 +1012,8 @@ typedef struct ProjectSetState
 	MemoryContext argcontext;	/* context for SRF arguments */
 } ProjectSetState;
 
+struct PartitionPruning;
+
 /* ----------------
  *	 ModifyTableState information
  * ----------------
@@ -1025,7 +1027,7 @@ typedef struct ModifyTableState
 	PlanState **mt_plans;		/* subplans (one per target rel) */
 	int			mt_nplans;		/* number of plans in the array */
 	int			mt_whichplan;	/* which one is being executed (0..n-1) */
-	ResultRelInfo *resultRelInfo;	/* per-subplan target relations */
+	ResultRelInfo **resultRelInfos;	/* per-subplan target relations */
 	ResultRelInfo *rootResultRelInfo;	/* root target relation (partitioned
 										 * table root) */
 	List	  **mt_arowmarks;	/* per-subplan ExecAuxRowMark lists */
@@ -1046,6 +1048,12 @@ typedef struct ModifyTableState
 
 	/* Per plan map for tuple conversion from child to root */
 	TupleConversionMap **mt_per_subplan_tupconv_maps;
+
+	/*
+	 * Details required to allow partitions to be eliminated from the scan, or
+	 * NULL if not possible.
+	 */
+	struct PartitionPruning *partition_pruning;
 } ModifyTableState;
 
 /* ----------------
@@ -1067,7 +1075,6 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
-struct PartitionPruning;
 
 struct AppendState
 {
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c90c763a86..d13e5117c9 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -238,6 +238,8 @@ typedef struct ModifyTable
 	Node	   *onConflictWhere;	/* WHERE for ON CONFLICT UPDATE */
 	Index		exclRelRTI;		/* RTI of the EXCLUDED pseudo relation */
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
+	List	   *part_prune_infos; /* Mapping details for run-time subplan
+								   * pruning, one per partitioned_rels */
 } ModifyTable;
 
 /* ----------------
-- 
2.16.2.windows.1

#96Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#95)
Re: [HACKERS] Runtime Partition Pruning

David Rowley wrote:

To put the new patch to the test, I tried pgbench -S -M prepared -s
100 with and without having modified pgbench_accounts to separate into
10 RANGE partitions of equal size.

A non-partitioned table was getting 12503 TPS.
With partitioned tables, the old version of this patch was getting: 5470 TPS.
With partitioned tables, the attached version gets 11247 TPS.
For perspective, today's master with a partitioned table gets 4719 TPS.

So you can see it's a pretty good performance boost by skipping
initialisation of the 9 non-matching subplans. It's not hard to
imagine the gains getting more significant with a larger number of
partitions.

These are excellent news!

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#97Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#95)
3 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 03/31/2018 09:52 AM, David Rowley wrote:

I've attached a new version of the patch. I'm now at v18 after having
some versions of the patch that I didn't release which were based on
various versions of Amit's faster partition pruning patch.

Thank you for the updated patch set !

I have tested this together with Amit's v46 patch.

The attached case doesn't trigger a generic plan, so basically all time
is spent in GetCachedPlan.

The standard table case (std.sql) gives:

generic_cost = 8.4525
avg_custom_cost = 13.4525
total_custom_cost = 67.2625

whereas the 64 hash partition case (hash.sql) gives:

generic_cost = 540.32
avg_custom_cost = 175.9425
total_custom_cost = 879.7125

I tested with pgbench -M prepared -f select.sql.

Also, I'm seeing a regression for check-world in
src/test/regress/results/inherit.out

***************
*** 642,648 ****
---------------------+---+---+-----
mlparted_tab_part1 | 1 | a |
mlparted_tab_part2a | 2 | a |
! mlparted_tab_part2b | 2 | b | xxx
mlparted_tab_part3 | 3 | a | xxx
(4 rows)

--- 642,648 ----
   ---------------------+---+---+-----
    mlparted_tab_part1  | 1 | a |
    mlparted_tab_part2a | 2 | a |
!  mlparted_tab_part2b | 2 | b |
    mlparted_tab_part3  | 3 | a | xxx
   (4 rows)

I'll spend some more time tomorrow.

Thanks for working on this !

Best regards,
Jesper

Attachments:

hash.sqlapplication/sql; name=hash.sqlDownload
select.sqlapplication/sql; name=select.sqlDownload
std.sqlapplication/sql; name=std.sqlDownload
#98Beena Emerson
memissemerson@gmail.com
In reply to: Jesper Pedersen (#97)
Re: [HACKERS] Runtime Partition Pruning

Hello,

On Tue, Apr 3, 2018 at 11:14 PM, Jesper Pedersen
<jesper.pedersen@redhat.com> wrote:

Hi David,

On 03/31/2018 09:52 AM, David Rowley wrote:

I've attached a new version of the patch. I'm now at v18 after having
some versions of the patch that I didn't release which were based on
various versions of Amit's faster partition pruning patch.

Thank you for the updated patch set !

I have tested this together with Amit's v46 patch.

Also, I'm seeing a regression for check-world in
src/test/regress/results/inherit.out

With Amit's v46 patch, the following query in partition_prune was
crashing during make check.
explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#99David Rowley
david.rowley@2ndquadrant.com
In reply to: Jesper Pedersen (#97)
Re: [HACKERS] Runtime Partition Pruning

On 4 April 2018 at 05:44, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

I have tested this together with Amit's v46 patch.

Thanks for testing this.

The attached case doesn't trigger a generic plan, so basically all time is
spent in GetCachedPlan.

Yeah, there's still no resolution to the fact that a generic plan +
runtime pruning might be cheaper than a custom plan. The problem is
the generic plan appears expensive to the custom vs generic plan
comparison due to it containing more Append subnodes and the run-time
pruning not being taking into account by that comparison.

There's been some discussion about this on this thread somewhere.

I think the best solution is probably the one suggested by Robert [1]/messages/by-id/CA+TgmoZv8sd9cKyYtHwmd_13+BAjkVKo=ECe7G98tBK5Ejwatw@mail.gmail.com
and that's to alter the Append plan's cost when run-time pruning is
enabled to try to account for the run-time pruning. This would be a
bit of a blind guess akin to what we do for clause selectivity
estimates for Params, but it's probably better than nothing, and
likely better than doing nothing.

Also, I'm seeing a regression for check-world in
src/test/regress/results/inherit.out

***************
*** 642,648 ****
---------------------+---+---+-----
mlparted_tab_part1 | 1 | a |
mlparted_tab_part2a | 2 | a |
! mlparted_tab_part2b | 2 | b | xxx
mlparted_tab_part3 | 3 | a | xxx
(4 rows)

--- 642,648 ----
---------------------+---+---+-----
mlparted_tab_part1  | 1 | a |
mlparted_tab_part2a | 2 | a |
!  mlparted_tab_part2b | 2 | b |
mlparted_tab_part3  | 3 | a | xxx
(4 rows)

I'll spend some more time tomorrow.

Yeah, it's a bug in v46 faster partition pruning. Discussing a fix for
that with Amit over on [2]/messages/by-id/CAKJS1f_6+gXB=Q+DryeB62yW7N19sY8hH_dBSjPFjm2ifdgoCw@mail.gmail.com.

If you patch v46 with the patch I attached to that thread, it should work.

[1]: /messages/by-id/CA+TgmoZv8sd9cKyYtHwmd_13+BAjkVKo=ECe7G98tBK5Ejwatw@mail.gmail.com
[2]: /messages/by-id/CAKJS1f_6+gXB=Q+DryeB62yW7N19sY8hH_dBSjPFjm2ifdgoCw@mail.gmail.com

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#100David Rowley
david.rowley@2ndquadrant.com
In reply to: Beena Emerson (#98)
Re: [HACKERS] Runtime Partition Pruning

On 4 April 2018 at 05:50, Beena Emerson <memissemerson@gmail.com> wrote:

With Amit's v46 patch, the following query in partition_prune was
crashing during make check.
explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);

Hi Beena,

Thanks for looking.

Does it work correctly if you apply [1]/messages/by-id/CAKJS1f_6+gXB=Q+DryeB62yW7N19sY8hH_dBSjPFjm2ifdgoCw@mail.gmail.com to Amit's v46 patch before
patching with v18 run-time partition pruning?

[1]: /messages/by-id/CAKJS1f_6+gXB=Q+DryeB62yW7N19sY8hH_dBSjPFjm2ifdgoCw@mail.gmail.com

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#101David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#99)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 4 April 2018 at 14:10, David Rowley <david.rowley@2ndquadrant.com> wrote:

On 4 April 2018 at 05:44, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

The attached case doesn't trigger a generic plan, so basically all time is
spent in GetCachedPlan.

Yeah, there's still no resolution to the fact that a generic plan +
runtime pruning might be cheaper than a custom plan. The problem is
the generic plan appears expensive to the custom vs generic plan
comparison due to it containing more Append subnodes and the run-time
pruning not being taking into account by that comparison.

Just for the record, some of the benchmarks I did above also used the
attached patch for the -M prepared case.

I didn't intend the patch for PostgreSQL, but I am starting to think
that it would be useful to have something to save from having to
EXECUTE PREPAREd statements 5 times before getting a generic plan.
Doing that is starting to seem a bit fragile to me. Would be nice to
have some solution, but I've so far not thought of anything better
than the attached (incomplete) patch.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Add-force_generic_plan-GUC.patchapplication/octet-stream; name=0001-Add-force_generic_plan-GUC.patchDownload
From 192e40895fd052291915994d9a0c051243daac32 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sat, 3 Mar 2018 21:14:38 +1300
Subject: [PATCH 1/4] Add force_generic_plan GUC

This, when set to on, bypasses the smarts which chooses a generic or custom
plan.  This can be used as a workaround if a PREPAREd statement insists on
using a custom plan each time.  Setting this to on will force the use of a
generic plan thus lowering the planner overhead.

This is intended for testing only as partition pruning cannot be performed
in the planner when the values being compared to the partition key are unknown
during planning. Run-time partition pruning improves this but still a custom
plan is often preferred due to it having a lower cost due to partitions having
been pruned.

This commit is not intended for inclusion in PostgreSQL.
---
 src/backend/utils/cache/plancache.c | 4 ++++
 src/backend/utils/misc/guc.c        | 9 +++++++++
 src/include/optimizer/cost.h        | 1 +
 3 files changed, 14 insertions(+)

diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 8d7d8e04c9..61c355e467 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -79,6 +79,8 @@
 	((plansource)->raw_parse_tree && \
 	 IsA((plansource)->raw_parse_tree->stmt, TransactionStmt))
 
+bool		force_generic_plan = false;
+
 /*
  * This is the head of the backend's list of "saved" CachedPlanSources (i.e.,
  * those that are in long-lived storage and are examined for sinval events).
@@ -1034,6 +1036,8 @@ choose_custom_plan(CachedPlanSource *plansource, ParamListInfo boundParams)
 	/* See if caller wants to force the decision */
 	if (plansource->cursor_options & CURSOR_OPT_GENERIC_PLAN)
 		return false;
+	if (force_generic_plan)
+		return false;
 	if (plansource->cursor_options & CURSOR_OPT_CUSTOM_PLAN)
 		return true;
 
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 1db7845d5a..a1e972c657 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -940,6 +940,15 @@ static struct config_bool ConfigureNamesBool[] =
 		true,
 		NULL, NULL, NULL
 	},
+	{
+		{"force_generic_plan", PGC_USERSET, QUERY_TUNING_METHOD,
+			gettext_noop("Forces PREPAREd statements to prefer the generic plan rather than a custom plan."),
+			NULL
+		},
+		&force_generic_plan,
+		false,
+		NULL, NULL, NULL
+	},
 	{
 		{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
 			gettext_noop("Enables genetic query optimization."),
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 132e35551b..14ec354516 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT bool enable_gathermerge;
 extern PGDLLIMPORT bool enable_partitionwise_join;
 extern PGDLLIMPORT bool enable_parallel_append;
 extern PGDLLIMPORT bool enable_parallel_hash;
+extern PGDLLIMPORT bool force_generic_plan;
 extern PGDLLIMPORT int	constraint_exclusion;
 
 extern double clamp_row_est(double nrows);
-- 
2.16.2.windows.1

#102Beena Emerson
memissemerson@gmail.com
In reply to: David Rowley (#100)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On Wed, Apr 4, 2018 at 7:57 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 4 April 2018 at 05:50, Beena Emerson <memissemerson@gmail.com> wrote:

With Amit's v46 patch, the following query in partition_prune was
crashing during make check.
explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);

Hi Beena,

Thanks for looking.

Does it work correctly if you apply [1] to Amit's v46 patch before
patching with v18 run-time partition pruning?

[1] /messages/by-id/CAKJS1f_6+gXB=Q+DryeB62yW7N19sY8hH_dBSjPFjm2ifdgoCw@mail.gmail.com

Thanks for working on it. make check passes when the patch [1] is also applied.

--

Beena Emerson

EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#103Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#99)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/04/04 11:10, David Rowley wrote:

On 4 April 2018 at 05:44, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

Also, I'm seeing a regression for check-world in
src/test/regress/results/inherit.out

***************
*** 642,648 ****
---------------------+---+---+-----
mlparted_tab_part1 | 1 | a |
mlparted_tab_part2a | 2 | a |
! mlparted_tab_part2b | 2 | b | xxx
mlparted_tab_part3 | 3 | a | xxx
(4 rows)

--- 642,648 ----
---------------------+---+---+-----
mlparted_tab_part1  | 1 | a |
mlparted_tab_part2a | 2 | a |
!  mlparted_tab_part2b | 2 | b |
mlparted_tab_part3  | 3 | a | xxx
(4 rows)

I'll spend some more time tomorrow.

Yeah, it's a bug in v46 faster partition pruning. Discussing a fix for
that with Amit over on [2].

I'm not sure if we've yet discussed anything that'd be related to this on
the faster pruning thread. It seems that the difference arises from
mlparted_tab_part2b not being selected for an update query that's executed
just before this test. When I execute an equivalent select query to check
if mlparted_tab_part2b is inadvertently pruned due to the new code, I
don't see the latest faster pruning patch doing it:

explain (costs off)
select *
from mlparted_tab mlp,
(select a from some_tab union all select a+1 from some_tab) ss (a)
where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
QUERY PLAN

----------------------------------------------------------------------------------
Nested Loop
Join Filter: (((mlp.a = some_tab.a) AND (mlp.b = 'b'::bpchar)) OR
(mlp.a = 3))
-> Append
-> Seq Scan on some_tab
-> Seq Scan on some_tab some_tab_1
-> Materialize
-> Append
-> Seq Scan on mlparted_tab_part1 mlp
Filter: ((b = 'b'::bpchar) OR (a = 3))
-> Seq Scan on mlparted_tab_part2b mlp_1
Filter: ((b = 'b'::bpchar) OR (a = 3))
-> Seq Scan on mlparted_tab_part3 mlp_2
Filter: ((b = 'b'::bpchar) OR (a = 3))
(13 rows)

For the original update query, constraint exclusion selects the same set
of partitions:

explain (costs off) update mlparted_tab mlp set c = 'xxx'
from (select a from some_tab union all select a+1 from some_tab) ss (a)
where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;
QUERY PLAN

----------------------------------------------------------------------------------------------
Update on mlparted_tab mlp
Update on mlparted_tab_part1 mlp_1
Update on mlparted_tab_part2b mlp_2
Update on mlparted_tab_part3 mlp_3
-> Nested Loop
Join Filter: (((mlp_1.a = some_tab.a) AND (mlp_1.b =
'b'::bpchar)) OR (mlp_1.a = 3))
-> Append
-> Seq Scan on some_tab
-> Seq Scan on some_tab some_tab_1
-> Materialize
-> Seq Scan on mlparted_tab_part1 mlp_1
Filter: ((b = 'b'::bpchar) OR (a = 3))
-> Nested Loop
Join Filter: (((mlp_2.a = some_tab.a) AND (mlp_2.b =
'b'::bpchar)) OR (mlp_2.a = 3))
-> Append
-> Seq Scan on some_tab
-> Seq Scan on some_tab some_tab_1
-> Materialize
-> Seq Scan on mlparted_tab_part2b mlp_2
Filter: ((b = 'b'::bpchar) OR (a = 3))
-> Nested Loop
Join Filter: (((mlp_3.a = some_tab.a) AND (mlp_3.b =
'b'::bpchar)) OR (mlp_3.a = 3))
-> Append
-> Seq Scan on some_tab
-> Seq Scan on some_tab some_tab_1
-> Materialize
-> Seq Scan on mlparted_tab_part3 mlp_3
Filter: ((b = 'b'::bpchar) OR (a = 3))
(28 rows)

What am I missing?

Thanks,
Amit

#104David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#103)
Re: [HACKERS] Runtime Partition Pruning

On 4 April 2018 at 18:27, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/04/04 11:10, David Rowley wrote:

On 4 April 2018 at 05:44, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

Also, I'm seeing a regression for check-world in
src/test/regress/results/inherit.out

***************
*** 642,648 ****
---------------------+---+---+-----
mlparted_tab_part1 | 1 | a |
mlparted_tab_part2a | 2 | a |
! mlparted_tab_part2b | 2 | b | xxx
mlparted_tab_part3 | 3 | a | xxx
(4 rows)

--- 642,648 ----
---------------------+---+---+-----
mlparted_tab_part1  | 1 | a |
mlparted_tab_part2a | 2 | a |
!  mlparted_tab_part2b | 2 | b |
mlparted_tab_part3  | 3 | a | xxx
(4 rows)

I'll spend some more time tomorrow.

Yeah, it's a bug in v46 faster partition pruning. Discussing a fix for
that with Amit over on [2].

I'm not sure if we've yet discussed anything that'd be related to this on
the faster pruning thread.

hmm, yeah, I didn't really explain the context, but the report was in [1]/messages/by-id/CAKJS1f_SHPuqDhQWJq-_P1kpPQn7BJt71yPbDP_8b3rhwFQyGA@mail.gmail.com

Basically, the OR clause in the following SQL fragment was overwriting
the scan_all_non_null value:

where (mlp.a = ss.a and mlp.b = 'b') or mlp.a = 3;

Basically the:

result->scan_all_nonnull = step_result->scan_all_nonnull;

The minimum fix would have been to change that line to:

result->scan_all_nonnull |= step_result->scan_all_nonnull;

Anyway, it all irrelevant now as that code has all changed.

[1]: /messages/by-id/CAKJS1f_SHPuqDhQWJq-_P1kpPQn7BJt71yPbDP_8b3rhwFQyGA@mail.gmail.com

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#105Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#104)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/04 16:04, David Rowley wrote:

On 4 April 2018 at 18:27, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

I'm not sure if we've yet discussed anything that'd be related to this on
the faster pruning thread.

hmm, yeah, I didn't really explain the context, but the report was in [1]>
[1] /messages/by-id/CAKJS1f_SHPuqDhQWJq-_P1kpPQn7BJt71yPbDP_8b3rhwFQyGA@mail.gmail.com

Oh, I see. Hopefully it is no longer an issue.

Thanks,
Amit

#106Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#99)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 04/03/2018 10:10 PM, David Rowley wrote:

The attached case doesn't trigger a generic plan, so basically all time is
spent in GetCachedPlan.

Yeah, there's still no resolution to the fact that a generic plan +
runtime pruning might be cheaper than a custom plan. The problem is
the generic plan appears expensive to the custom vs generic plan
comparison due to it containing more Append subnodes and the run-time
pruning not being taking into account by that comparison.

There's been some discussion about this on this thread somewhere.

Forgot about that, sorry.

I think the best solution is probably the one suggested by Robert [1]
and that's to alter the Append plan's cost when run-time pruning is
enabled to try to account for the run-time pruning. This would be a
bit of a blind guess akin to what we do for clause selectivity
estimates for Params, but it's probably better than nothing, and
likely better than doing nothing.

Yeah, something based on the number of WHERE clauses, and if the
partition type has DEFAULT / NULL partition could help. Forcing
choose_custom_plan() to return false does help a lot (> 400%) for the
HASH case.

But maybe this area is best left for PG12.

Yeah, it's a bug in v46 faster partition pruning. Discussing a fix for
that with Amit over on [2].

I was running with a version of faster_part_prune_v45_fixups.patch.

Patch v49 with v18 (0001-0004) works. 0005 needs a rebase.

Thanks again,
Jesper

#107David Rowley
david.rowley@2ndquadrant.com
In reply to: Jesper Pedersen (#106)
5 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 5 April 2018 at 05:31, Jesper Pedersen <jesper.pedersen@redhat.com> wrote:

Patch v49 with v18 (0001-0004) works. 0005 needs a rebase.

I've attached an updated patchset against Amit's v49 faster partition
pruning patch [1]/messages/by-id/c5331ff6-8b31-0742-758e-bd7b9aeddf07@lab.ntt.co.jp.

v49 should also be patched with [2]/messages/by-id/CAKJS1f_ad=B9rVf5dPD27=xTN1Ob7xJi6N4BFBNytj93rCPqzg@mail.gmail.com and [3]/messages/by-id/3eedafaa-840f-bf72-2fa9-dadb0852e959@redhat.com.

I'm pretty happy with patches 0001 to 0004. At the moment I'm still a
bit unsure of 0005. I need to do a bit more sanity checking on it,
mostly around the changes in planner.c. Although I am keen to see 0005
make it into PG11 as it does make running larger numbers of partition
more usable for UPDATE/DELETE too. Amit's improvements are really good
alone, but unfortunately, anyone who's waiting on us speeding up the
partition pruning in SELECT queries is probably also waiting on us
doing the same for UPDATE/DELETE. So I do think 0005 is important for
PG11, providing it's correct, of course.

[1]: /messages/by-id/c5331ff6-8b31-0742-758e-bd7b9aeddf07@lab.ntt.co.jp
[2]: /messages/by-id/CAKJS1f_ad=B9rVf5dPD27=xTN1Ob7xJi6N4BFBNytj93rCPqzg@mail.gmail.com
[3]: /messages/by-id/3eedafaa-840f-bf72-2fa9-dadb0852e959@redhat.com

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v19-0001-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v19-0001-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From d8f13e3426dd69c908a1d665463ab6ee250bf8bb Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 5 Apr 2018 11:55:03 +1200
Subject: [PATCH v19 1/5] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/catalog/partition.c        |  23 ++
 src/backend/commands/explain.c         |  51 ++--
 src/backend/executor/execPartition.c   | 466 +++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c          |  20 ++
 src/backend/nodes/outfuncs.c           |  27 ++
 src/backend/nodes/readfuncs.c          |  19 ++
 src/backend/optimizer/util/partprune.c | 257 ++++++++++++++++++
 src/include/catalog/partition.h        |  13 +
 src/include/executor/execPartition.h   |  77 ++++++
 src/include/nodes/nodes.h              |   1 +
 src/include/nodes/primnodes.h          |  23 ++
 src/include/optimizer/partprune.h      |   4 +
 12 files changed, 963 insertions(+), 18 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index d6bce9f348..2f562df81f 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1991,6 +1991,29 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+			/*
+			 * 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;
+				bool isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+											context->planstate->ps_ExprContext,
+												   &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+
+			}
 		default:
 			break;
 	}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 79f639d5e2..fa86212769 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1811,28 +1811,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3173,18 +3178,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that
+	 * was specified in the plan means that some subnodes have been ignored
+	 * per instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index a6a7885abd..17da8cdbd3 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,6 +39,12 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_extparams_recurse(
+									PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans);
+static void find_subplans_for_allparams_recurse(
+									PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans);
 
 
 /*
@@ -1267,3 +1273,463 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all. This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruning:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruning details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruning
+ *
+ * Setup the required data structure for calling ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These context can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionedRelPruning *partrelprunes;
+	PartitionPruning *partprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	partprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionedRelPruning *)
+					palloc(sizeof(PartitionedRelPruning) *
+						   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	partprune->partrelpruning = partrelprunes;
+	partprune->npartrelpruning = list_length(partitionpruneinfo);
+	partprune->extparams = NULL;
+	partprune->execparams = NULL;
+	partprune->allparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	partprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+													 "Partition Prune",
+													 ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionedRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc		partdesc;
+		Relation			rel;
+		PartitionKey		partkey;
+		int					partnatts;
+		int		j;
+
+		partrelprune->allpartindexes = bms_copy(pinfo->allpartindexes);
+		partrelprune->nparts = pinfo->nparts;
+		partrelprune->subnodeindex = palloc(sizeof(int) * pinfo->nparts);
+		partrelprune->subpartprune = palloc(sizeof(PartitionedRelPruning *) *
+										 pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(partrelprune->subnodeindex, pinfo->subnodeindex,
+			   sizeof(int) * pinfo->nparts);
+
+		for (j = 0; j < pinfo->nparts; j++)
+		{
+			int subpartidx = pinfo->subpartindex[j];
+
+			Assert(subpartidx < list_length(partitionpruneinfo));
+
+			if (subpartidx >= 0)
+				partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
+			else
+				partrelprune->subpartprune[j] = NULL;
+		}
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL;		/* empty for now */
+
+		partrelprune->prunesteps = pinfo->prunesteps;
+
+		partrelprune->extparams = bms_copy(pinfo->extparams);
+		partrelprune->allparams = bms_union(pinfo->extparams,
+											pinfo->execparams);
+
+		partprune->extparams = bms_add_members(partprune->extparams,
+											   pinfo->extparams);
+
+		partprune->execparams = bms_add_members(partprune->execparams,
+												pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the Param ids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	partprune->allparams = bms_union(partprune->extparams,
+									 partprune->execparams);
+
+	return partprune;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'partprune'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so once during their init
+ * plan.  The caller must only initialize the subnodes which are returned by
+ * this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'partprune' has any extparams.
+ *
+ * 'nsubplans' must be passed as the total number of unpruned subplans.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
+{
+	PartitionedRelPruning *partrelprune;
+	MemoryContext		oldcontext;
+	Bitmapset		   *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(partprune->extparams));
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	/* Determine which subplans match these external params */
+	find_subplans_for_extparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been done for external Params.
+	 */
+	bms_free(partprune->extparams);
+	partprune->extparams = NULL;
+
+	/*
+	 * If any subplans were pruned, we must re-sequence the subplan indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subplans which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubplans)
+	{
+		int	   *subplanidxmap;
+		int		i;
+		int		newidx;
+
+		/*
+		 * First we must build a map which allows us to map the old subplan
+		 * index into the new one.
+		 */
+		subplanidxmap = (int *) palloc(sizeof(int) * nsubplans);
+		newidx = 0;
+		for (i = 0; i < nsubplans; i++)
+		{
+			if (bms_is_member(i, result))
+				subplanidxmap[i] = newidx++;
+			else
+				subplanidxmap[i] = -1;		/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnodeindex
+		 * so that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < partprune->npartrelpruning; i++)
+		{
+			PartitionedRelPruning *partrelprune;
+			int j;
+
+			partrelprune = &partprune->partrelpruning[i];
+
+			/*
+			 * We also need to reset the allpartindexes field so that it
+			 * only contains partition indexes that we actually still have
+			 * subnodeindexes for.  It seems easier to build a fresh one,
+			 * rather than trying to update the existing one.
+			 */
+			bms_free(partrelprune->allpartindexes);
+			partrelprune->allpartindexes = NULL;
+
+			for (j = 0; j < partrelprune->nparts; j++)
+			{
+				int oldidx = partrelprune->subnodeindex[j];
+
+				/*
+				 * If this partition existed as a subplan then change the old
+				 * subplan index to the new subplan index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subplan list due to some subplans
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					partrelprune->subnodeindex[j] = subplanidxmap[oldidx];
+
+					if (subplanidxmap[oldidx] >= 0)
+						partrelprune->allpartindexes =
+								bms_add_member(partrelprune->allpartindexes,
+											   j);
+				}
+			}
+		}
+
+		pfree(subplanidxmap);
+	}
+
+
+	return result;
+}
+
+/*
+ * find_subplans_for_extparams_recurse
+ *		Recursive worker function for ExecFindInitialMatchingSubPlans.
+ */
+static void
+find_subplans_for_extparams_recurse(PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset	   *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * We only need to determine the matching partitions if there are any
+	 * ext params matching the partition key at this level.  If there are no
+	 * such 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.
+	 */
+	if (!bms_is_empty(partrelprune->extparams))
+	{
+		context->safeparams = partrelprune->extparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_extparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'partprune' for the current Param values.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *partprune)
+{
+	PartitionedRelPruning *partrelprune;
+	MemoryContext		oldcontext;
+	Bitmapset *result = NULL;
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the
+	 * executor's memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	find_subplans_for_allparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_allparams_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans.
+ */
+static void
+find_subplans_for_allparams_recurse(PartitionedRelPruning *partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset	   *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(partrelprune->allparams))
+	{
+		context->safeparams = partrelprune->allparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_allparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b0fa556f71..96501b5ba5 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2167,6 +2167,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(prunesteps);
+	COPY_BITMAPSET_FIELD(allpartindexes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5103,6 +5120,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e6793b4716..ea7c4b7ac6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1746,6 +1746,30 @@ _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(prunesteps);
+	WRITE_BITMAPSET_FIELD(allpartindexes);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnodeindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnodeindex[i]);
+
+	appendStringInfoString(str, " :subpartindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpartindex[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3977,6 +4001,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 25874074a0..b3a4f45da4 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1357,6 +1357,23 @@ _readPartitionPruneStepCombine(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(prunesteps);
+	READ_BITMAPSET_FIELD(allpartindexes);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnodeindex, local_node->nparts);
+	READ_INT_ARRAY(subpartindex, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2626,6 +2643,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 75b7232f5d..db9b7773cf 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -27,6 +27,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -81,6 +82,7 @@ typedef struct GeneratePruningStepsContext
 	List   *steps;
 }			GeneratePruningStepsContext;
 
+static bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static List *generate_partition_pruning_steps_internal(RelOptInfo *rel,
 									  GeneratePruningStepsContext *context,
 									  List *clauses,
@@ -166,6 +168,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 		context.nparts = rel->nparts;
 		context.boundinfo = rel->boundinfo;
 
+		/* Not valid when being called from the planner */
+		context.planstate = NULL;
+		context.safeparams = NULL;
+
 		partindexes = get_matching_partitions(&context, pruning_steps);
 
 		/* Add selected partitions' RT indexes to result. */
@@ -229,8 +235,259 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 	return context.steps;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Return a List of PartitionPruneInfos, one for each 'partitioned_rel',
+ *		or NIL if no Params were found matching the partition key, in which
+ *		case run-time partition pruning is useless.
+ *
+ * Here we index the subpaths by partition index so that we're able to
+ * translate the output of get_matching_partitions into subpath indexes to
+ * possibly allow for further partition pruning to be performed during
+ * execution.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal)
+{
+	RangeTblEntry	   *rte;
+	RelOptInfo		   *parentpart;
+	ListCell		   *lc;
+	List			   *pinfolist = NIL;
+	int				   *allsubnodeindex;
+	int				   *allsubpartindex;
+	int					i;
+	bool				gotparam = false;
+
+	/*
+	 * Allocate two arrays, one to allow quick lookups of the 'subpaths' index
+	 * of a relation by relid and another to lookup the 'partitioned_rel'
+	 * index by relid.
+	 */
+	allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
+	allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);
+
+	/* Initialize to -1 to indicate the rel was not found */
+	for (i = 0; i < root->simple_rel_array_size; i++)
+	{
+		allsubnodeindex[i] = -1;
+		allsubpartindex[i] = -1;
+	}
+
+	/*
+	 * Now loop over each subpath and fill in the index of the subpath for the
+	 * subpath's relid.
+	 */
+	if (resultRelations != NIL)
+	{
+		i = 0;
+		foreach(lc, resultRelations)
+		{
+			int resultrel = lfirst_int(lc);
+			Assert(resultrel < root->simple_rel_array_size);
+			allsubnodeindex[resultrel] = i;
+			i++;
+		}
+	}
+	else
+	{
+		i = 0;
+		foreach(lc, subpaths)
+		{
+			Path *path = (Path *) lfirst(lc);
+			RelOptInfo *pathrel = path->parent;
+
+			Assert(IS_SIMPLE_REL(pathrel));
+			Assert(pathrel->relid < root->simple_rel_array_size);
+
+			allsubnodeindex[pathrel->relid] = i;
+			i++;
+		}
+	}
+
+	/* Likewise for the partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index	rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		allsubpartindex[rti] = i;
+		i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		int			nparts = subpart->nparts;
+		int		   *subnodeindex;
+		int		   *subpartindex;
+		List	   *partprunequal;
+		bool		constfalse;
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (i == 0)
+		{
+			parentpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+						adjust_appendrel_attrs_multilevel(root,
+														  (Node *) prunequal,
+														  subpart->relids,
+														  parentpart->relids);
+		}
+
+		pinfo->prunesteps = generate_partition_pruning_steps(subpart,
+															 partprunequal,
+															 &constfalse);
+
+		if (constfalse)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered.  We'd
+			 * better do something sane here anyway, so let's just disable
+			 * run-time pruning.
+			 */
+			return NIL;
+		}
+
+		pinfo->allpartindexes = NULL;
+		pinfo->nparts = nparts;
+		pinfo->subnodeindex = subnodeindex = palloc(nparts * sizeof(int));
+		pinfo->subpartindex = subpartindex = palloc(nparts * sizeof(int));
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+
+		/*
+		 * 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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pinfo->prunesteps);
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present
+		 * in the subpaths List will be set to -1, and any subpartitioned
+		 * table which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = allsubnodeindex[partrel->relid];
+			int			subpartidx = allsubpartindex[partrel->relid];
+
+			subnodeindex[i] = subnodeidx;
+			subpartindex[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				pinfo->allpartindexes = bms_add_member(pinfo->allpartindexes,
+													   i);
+		}
+
+		pinfolist = lappend(pinfolist, pinfo);
+		i++;
+	}
+
+	pfree(allsubnodeindex);
+	pfree(allsubpartindex);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
+
 /* Module-local functions */
 
+/*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							(int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
 /*
  * generate_partition_pruning_steps_internal
  *		Processes 'clauses' to generate partition pruning steps.
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index 8981901272..558a996ac3 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -62,6 +63,18 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow
+	 * resolution of Param values.
+	 */
+	PlanState	*planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until after init plan.
+	 */
+	Bitmapset *safeparams;
 } PartitionPruneContext;
 
 extern void RelationBuildPartitionDesc(Relation relation);
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 9f55f6409e..140e0bdf1e 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "optimizer/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,77 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionedRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows partprune.c's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'prunesteps' will be found within.
+ *
+ * nparts						The number of partitions which belong to this
+ *								partitioned relation. Also defines the size of
+ *								the 'subnodeindex' and 'subpartprune' arrays.
+ * subnodeindex					An array of nparts containing the subnode
+ *								index which matches this partition index, or
+ *								-1 if there is no match.
+ * subpartprune					An array of nparts containing the
+ *								PartitionedRelPruning details this partition
+ *								index for sub-partitioned tables.
+ * allpartindexes				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ *								belong to this partition.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * prunesteps					Contains list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ *-----------------------
+ */
+typedef struct PartitionedRelPruning
+{
+	int			nparts;
+	int		   *subnodeindex;
+	struct PartitionedRelPruning **subpartprune;
+	Bitmapset  *allpartindexes;
+	PartitionPruneContext context;
+	List	   *prunesteps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionedRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionedRelPruning
+ * structs and also stores all Param IDs which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partrelpruning		Array of PartitionedRelPruning for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * npartrelpruning		Number of items in partrelpruning array.
+ * prune_context		A memory context which can be used to call the query
+ *						planner's partition prune functions.
+ * extparams			All PARAM_EXTERN Param IDs which were found to match a
+ *						partition key in each of the contained
+ *						PartitionedRelPruning structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of extparams and execparams, saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionedRelPruning *partrelpruning;
+	int npartrelpruning;
+	MemoryContext prune_context;
+	Bitmapset *extparams;
+	Bitmapset *execparams;
+	Bitmapset *allparams;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -127,5 +199,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+								int nsubplans);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 1ec8030d4b..940469e41a 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -196,6 +196,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 965eb656a8..f357473c6a 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1579,4 +1579,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;		/* Oid of partition rel */
+	List	   *prunesteps;	/* List of PartitionPruneStep */
+	Bitmapset  *allpartindexes;	/* All part index we have subnodes for at this
+								 * level */
+	int			nparts;		/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+	int		   *subpartindex;	/* subpart index indexed by partition id */
+	Bitmapset  *extparams;		/* All external ParamIDs seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec ParamIDs seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 1f2fe297a3..b7352d150c 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -20,4 +20,8 @@ extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
-- 
2.16.2.windows.1

v19-0002-Add-bms_prev_member-function.patchapplication/octet-stream; name=v19-0002-Add-bms_prev_member-function.patchDownload
From 5b485f5307ca3cd99864d2e9bcc691fcaf9fd43a Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 5 Apr 2018 11:55:55 +1200
Subject: [PATCH v19 2/5] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1088,6 +1110,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 	return -2;
 }
 
+/*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
 /*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.16.2.windows.1

v19-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchapplication/octet-stream; name=v19-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchDownload
From e523e9f644c676587afccf080415078c88a3513d Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 5 Apr 2018 11:57:16 +1200
Subject: [PATCH v19 3/5] Allow Append subnodes to be pruned during execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  258 ++++--
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   15 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1800 insertions(+), 88 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..5286ada835 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +101,9 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i, j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -112,55 +115,117 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+	appendstate->as_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&appendstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set a noop flag so that we never actually
+			 * bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+							bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+												 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
-	 */
-	i = 0;
+		* call ExecInitNode on each of the valid plans to be executed and save
+		* the results into the appendplanstates array.
+		*/
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
-
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
+
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +316,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +348,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +438,40 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/* Handle the case for when all subplans were pruned */
+	if (node->as_noopscan)
+		return false;
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -411,10 +500,33 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
 	else
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +579,25 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
+
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +663,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 96501b5ba5..ee4c70aef8 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -248,6 +248,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index c3f1789ce2..db1470ce16 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3802,32 +3802,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3877,15 +3877,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index ea7c4b7ac6..e22ad4d4dd 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -418,6 +418,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index b3a4f45da4..2e22da6c4f 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1675,6 +1675,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index c36a254ed6..25cfa5e349 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1607,7 +1607,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1649,8 +1649,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1698,7 +1698,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1761,7 +1761,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2027,7 +2027,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99d0736029..76fe80fbb7 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1041,6 +1042,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1078,6 +1081,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels, NIL,
+											best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1086,7 +1120,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5382,7 +5417,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5394,7 +5430,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 008492bad5..421dc79cc4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3920,7 +3920,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6852,8 +6853,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 67e47887fc..2ce4d4496d 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 416b3f9578..d585b6b7d2 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+																  rel,
+																  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3574,7 +3592,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ff63d179b2..5890209fee 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1114,8 +1114,15 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
+ *		noopscan			true if partition pruning proved that none of the
+ *							appendplans can contain a record to satisfy this
+ *							query.
  * ----------------
  */
 
@@ -1123,6 +1130,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruning;
 
 struct AppendState
 {
@@ -1132,7 +1140,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset   *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
+	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0a797f0a05..c3e5c2c79f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -256,6 +256,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 895bf6959d..4f65686d9b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 2d77b3edd4..0a2517125c 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1516,3 +1516,1138 @@ explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 (9 rows)
 
 drop table hp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index ad5177715c..eefbf32e93 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -274,3 +274,347 @@ explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and
 explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 
 drop table hp;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.16.2.windows.1

v19-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchapplication/octet-stream; name=v19-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchDownload
From 221cfd2e067cda6dc13dda3b0c0627bfd215f0fb Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 5 Apr 2018 11:57:58 +1200
Subject: [PATCH v19 4/5] Allow MergeAppend's subnodes to be pruned during
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   4 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 351 insertions(+), 22 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..d9cf911f4f 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,9 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i, j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -78,19 +80,89 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mergestate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+							bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +173,23 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
-
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +247,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+						ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +374,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ee4c70aef8..739a023965 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -276,6 +276,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e22ad4d4dd..e31b6a9c33 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -450,6 +450,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2e22da6c4f..5bf3d28c51 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1697,6 +1697,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 76fe80fbb7..d6c94846d3 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1144,6 +1144,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1229,8 +1231,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels, NIL,
+											best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5890209fee..9851f35f77 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1155,6 +1155,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1167,6 +1173,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset   *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c3e5c2c79f..0b5189aa7d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -280,6 +280,10 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 0a2517125c..792924fe0b 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2651,3 +2651,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index eefbf32e93..6f389bed1f 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -618,3 +618,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.16.2.windows.1

v19-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchapplication/octet-stream; name=v19-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchDownload
From da4f376a16d2fbde48900b654147f251782f3a76 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 5 Apr 2018 12:53:02 +1200
Subject: [PATCH v19 5/5] Improve planning speed of partitioned table
 UPDATE/DELETEs

By making a call to grouping_planner for the complete parse of the query we
can make use of the faster partition pruning code used there.  This will
identify all partitions which could be pruned as IS_DUMMY_RELs, of which we
can skip performing each individual grouping_planner call inside
inheritance_planner.  This can improve planner performance significantly when
there are many partitions. There may be a slight slowdown when no partitions
could be pruned or when there are very few (1 or 2) partitions.  However it
seems better to optimize the case when partitions are pruned, rather than the
case where they're not, as those queries are less likely to be fast to
execute. The case for partitioned tables with just 1 or 2 leaf partitions does
not seem worth worrying about too much. The measured regression on 1 partition
was just 10% of overall planning time.

This commit also implements run-time partition pruning for UPDATE/DELETE.
---
 src/backend/commands/explain.c          |   4 +-
 src/backend/executor/execPartition.c    |  18 ++--
 src/backend/executor/nodeMerge.c        |   4 +-
 src/backend/executor/nodeModifyTable.c  | 160 +++++++++++++++++++++++++-------
 src/backend/nodes/copyfuncs.c           |   1 +
 src/backend/nodes/outfuncs.c            |   1 +
 src/backend/nodes/readfuncs.c           |   1 +
 src/backend/optimizer/plan/createplan.c |  32 ++++++-
 src/backend/optimizer/plan/planner.c    |  59 ++++++++++++
 src/include/nodes/execnodes.h           |  11 ++-
 src/include/nodes/plannodes.h           |   2 +
 11 files changed, 239 insertions(+), 54 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index fa86212769..c593748d84 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -3028,14 +3028,14 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 	/* Should we explicitly label target relations? */
 	labeltargets = (mtstate->mt_nplans > 1 ||
 					(mtstate->mt_nplans == 1 &&
-					 mtstate->resultRelInfo->ri_RangeTableIndex != node->nominalRelation));
+					 mtstate->resultRelInfos[0]->ri_RangeTableIndex != node->nominalRelation));
 
 	if (labeltargets)
 		ExplainOpenGroup("Target Tables", "Target Tables", false, es);
 
 	for (j = 0; j < mtstate->mt_nplans; j++)
 	{
-		ResultRelInfo *resultRelInfo = mtstate->resultRelInfo + j;
+		ResultRelInfo *resultRelInfo = mtstate->resultRelInfos[j];
 		FdwRoutine *fdwroutine = resultRelInfo->ri_FdwRoutine;
 
 		if (labeltargets)
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 17da8cdbd3..1e67308a7a 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -103,7 +103,7 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
 
 	if (is_update)
 	{
-		update_rri = mtstate->resultRelInfo;
+		update_rri = mtstate->resultRelInfos[0];
 		num_update_rri = list_length(node->plans);
 		proute->subplan_partition_offsets =
 			palloc(num_update_rri * sizeof(int));
@@ -418,8 +418,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * In the case of INSERT on a partitioned table, there is only one
@@ -474,8 +474,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		TupleTableSlot *slot;
 		ExprContext *econtext;
 		List	   *returningList;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/* See the comment above for WCO lists. */
 		Assert((node->operation == CMD_INSERT &&
@@ -530,8 +530,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	if (node && node->onConflictAction != ONCONFLICT_NONE)
 	{
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		ExprContext *econtext = mtstate->ps.ps_ExprContext;
 		ListCell   *lc;
@@ -671,8 +671,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	{
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * If the root parent and partition have the same tuple
diff --git a/src/backend/executor/nodeMerge.c b/src/backend/executor/nodeMerge.c
index 0e0d0795d4..29d8c544a2 100644
--- a/src/backend/executor/nodeMerge.c
+++ b/src/backend/executor/nodeMerge.c
@@ -101,7 +101,7 @@ ExecMergeMatched(ModifyTableState *mtstate, EState *estate,
 		if (resultRelInfo == NULL)
 		{
 			resultRelInfo = ExecInitPartitionInfo(mtstate,
-					mtstate->resultRelInfo,
+					mtstate->resultRelInfos[0],
 					proute, estate, leaf_part_index);
 			Assert(resultRelInfo != NULL);
 		}
@@ -397,7 +397,7 @@ ExecMergeNotMatched(ModifyTableState *mtstate, EState *estate,
 	 * currently active result relation, which should be of the root of the
 	 * partition tree.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 
 	/*
 	 * For INSERT actions, root relation's merge action is OK since the
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index b03db64e8e..adf09b9609 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -1219,12 +1219,12 @@ lreplace:;
 				map_index = resultRelInfo->ri_PartitionLeafIndex;
 				Assert(mtstate->rootResultRelInfo == NULL);
 				tupconv_map = TupConvMapForLeaf(proute,
-								mtstate->resultRelInfo,
+								mtstate->resultRelInfos[0],
 								map_index);
 			}
 			else
 			{
-				map_index = resultRelInfo - mtstate->resultRelInfo;
+				map_index = mtstate->mt_whichplan;
 				Assert(map_index >= 0 && map_index < mtstate->mt_nplans);
 				tupconv_map = tupconv_map_for_subplan(mtstate, map_index);
 			}
@@ -1637,12 +1637,12 @@ static void
 fireBSTriggers(ModifyTableState *node)
 {
 	ModifyTable *plan = (ModifyTable *) node->ps.plan;
-	ResultRelInfo *resultRelInfo = node->resultRelInfo;
+	ResultRelInfo *resultRelInfo = node->resultRelInfos[0];
 
 	/*
 	 * If the node modifies a partitioned table, we must fire its triggers.
-	 * Note that in that case, node->resultRelInfo points to the first leaf
-	 * partition, not the root table.
+	 * Note that in that case, node->resultRelInfos[0] points to the first
+	 * leaf partition, not the root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		resultRelInfo = node->rootResultRelInfo;
@@ -1688,13 +1688,14 @@ static ResultRelInfo *
 getTargetResultRelInfo(ModifyTableState *node)
 {
 	/*
-	 * Note that if the node modifies a partitioned table, node->resultRelInfo
-	 * points to the first leaf partition, not the root table.
+	 * Note that if the node modifies a partitioned table,
+	 * node->resultRelInfos[0] points to the first leaf partition, not the
+	 * root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		return node->rootResultRelInfo;
 	else
-		return node->resultRelInfo;
+		return node->resultRelInfos[0];
 }
 
 /*
@@ -1915,7 +1916,7 @@ static void
 ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 {
 	ResultRelInfo *targetRelInfo = getTargetResultRelInfo(mtstate);
-	ResultRelInfo *resultRelInfos = mtstate->resultRelInfo;
+	ResultRelInfo **resultRelInfos = mtstate->resultRelInfos;
 	TupleDesc	outdesc;
 	int			numResultRelInfos = mtstate->mt_nplans;
 	int			i;
@@ -1946,7 +1947,7 @@ ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 	for (i = 0; i < numResultRelInfos; ++i)
 	{
 		mtstate->mt_per_subplan_tupconv_maps[i] =
-			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i].ri_RelationDesc),
+			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i]->ri_RelationDesc),
 								   outdesc,
 								   gettext_noop("could not convert row type"));
 	}
@@ -2085,7 +2086,7 @@ ExecModifyTable(PlanState *pstate)
 	}
 
 	/* Preload local variables */
-	resultRelInfo = node->resultRelInfo + node->mt_whichplan;
+	resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 	subplanstate = node->mt_plans[node->mt_whichplan];
 	junkfilter = resultRelInfo->ri_junkFilter;
 
@@ -2123,7 +2124,7 @@ ExecModifyTable(PlanState *pstate)
 
 			if (node->mt_whichplan < node->mt_nplans)
 			{
-				resultRelInfo++;
+				resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 				subplanstate = node->mt_plans[node->mt_whichplan];
 				junkfilter = resultRelInfo->ri_junkFilter;
 				estate->es_result_relation_info = resultRelInfo;
@@ -2314,9 +2315,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	ResultRelInfo *resultRelInfo;
 	Plan	   *subplan;
 	ListCell   *l;
-	int			i;
+	int			i, j;
 	Relation	rel;
 	bool		update_tuple_routing_needed = node->partColsUpdated;
+	Bitmapset  *validsubplans;
 
 	/* check for unsupported flags */
 	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -2333,8 +2335,70 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	mtstate->canSetTag = node->canSetTag;
 	mtstate->mt_done = false;
 
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mtstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mtstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away ModifyTable plans.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+											list_length(node->plans));
+
+			/*
+			 * If no plans match the given parameters then we must handle this
+			 * case in a special way.  The problem here is that code in
+			 * explain.c requires a ModifyTable to have at least one plan in
+			 * order for it to properly determine the Vars in that plan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set the mt_done flag so that we never
+			 * actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mtstate->mt_done = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->plans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		mtstate->partition_pruning = partprune;
+	}
+	else
+	{
+		nplans = list_length(node->plans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark
+		 * all plans as valid, they must also all be initialized.
+		 */
+		validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		mtstate->partition_pruning = NULL;
+	}
+
+
 	mtstate->mt_plans = (PlanState **) palloc0(sizeof(PlanState *) * nplans);
-	mtstate->resultRelInfo = estate->es_result_relations + node->resultRelIndex;
+	mtstate->resultRelInfos = (ResultRelInfo **)
+									palloc(sizeof(ResultRelInfo *) * nplans);
 
 	/* If modifying a partitioned table, initialize the root table info */
 	if (node->rootResultRelIndex >= 0)
@@ -2358,8 +2422,6 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	 */
 	saved_resultRelInfo = estate->es_result_relation_info;
 
-	resultRelInfo = mtstate->resultRelInfo;
-
 	/*
 	 * mergeTargetRelation must be set if we're running MERGE and mustn't be
 	 * set if we're not.
@@ -2367,13 +2429,20 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	Assert(operation != CMD_MERGE || node->mergeTargetRelation > 0);
 	Assert(operation == CMD_MERGE || node->mergeTargetRelation == 0);
 
-	resultRelInfo->ri_mergeTargetRTI = node->mergeTargetRelation;
-
-	i = 0;
+	j = i = 0;
 	foreach(l, node->plans)
 	{
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
 		subplan = (Plan *) lfirst(l);
 
+		resultRelInfo = estate->es_result_relations + node->resultRelIndex + i;
+		mtstate->resultRelInfos[j] = resultRelInfo;
+
 		/* Initialize the usesFdwDirectModify flag */
 		resultRelInfo->ri_usesFdwDirectModify = bms_is_member(i,
 															  node->fdwDirectModifyPlans);
@@ -2410,7 +2479,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		/* Now init the plan for this result rel */
 		estate->es_result_relation_info = resultRelInfo;
-		mtstate->mt_plans[i] = ExecInitNode(subplan, estate, eflags);
+		mtstate->mt_plans[j] = ExecInitNode(subplan, estate, eflags);
 
 		/* Also let FDWs init themselves for foreign-table result rels */
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
@@ -2426,10 +2495,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 															 eflags);
 		}
 
-		resultRelInfo++;
 		i++;
+		j++;
 	}
 
+	mtstate->resultRelInfos[0]->ri_mergeTargetRTI = node->mergeTargetRelation;
+
 	estate->es_result_relation_info = saved_resultRelInfo;
 
 	/* Get the target relation */
@@ -2482,26 +2553,34 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	/*
 	 * Initialize any WITH CHECK OPTION constraints if needed.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
-	i = 0;
+	j = i = 0;
 	foreach(l, node->withCheckOptionLists)
 	{
-		List	   *wcoList = (List *) lfirst(l);
+		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
 
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
+		wcoList = (List *) lfirst(l);
+
 		foreach(ll, wcoList)
 		{
 			WithCheckOption *wco = (WithCheckOption *) lfirst(ll);
 			ExprState  *wcoExpr = ExecInitQual((List *) wco->qual,
-											   mtstate->mt_plans[i]);
+											   mtstate->mt_plans[j]);
 
 			wcoExprs = lappend(wcoExprs, wcoExpr);
 		}
-
+		resultRelInfo = mtstate->resultRelInfos[j];
 		resultRelInfo->ri_WithCheckOptions = wcoList;
 		resultRelInfo->ri_WithCheckOptionExprs = wcoExprs;
-		resultRelInfo++;
+
+		j++;
 		i++;
 	}
 
@@ -2531,15 +2610,25 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		/*
 		 * Build a projection for each result rel.
 		 */
-		resultRelInfo = mtstate->resultRelInfo;
+		j = i = 0;
 		foreach(l, node->returningLists)
 		{
-			List	   *rlist = (List *) lfirst(l);
+			List	   *rlist;
+
+			if (!bms_is_member(i, validsubplans))
+			{
+				i++;
+				continue;
+			}
+
+			rlist  = (List *) lfirst(l);
 
+			resultRelInfo = mtstate->resultRelInfos[j];
 			resultRelInfo->ri_projectReturning =
 				ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps,
 										resultRelInfo->ri_RelationDesc->rd_att);
-			resultRelInfo++;
+			j++;
+			i++;
 		}
 	}
 	else
@@ -2555,7 +2644,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	}
 
 	/* Set the list of arbiter indexes if needed for ON CONFLICT */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (node->onConflictAction != ONCONFLICT_NONE)
 		resultRelInfo->ri_onConflictArbiterIndexes = node->arbiterIndexes;
 
@@ -2659,7 +2748,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		}
 	}
 
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 
 	if (node->mergeActionList)
 	{
@@ -2810,11 +2899,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		if (junk_filter_needed)
 		{
-			resultRelInfo = mtstate->resultRelInfo;
 			for (i = 0; i < nplans; i++)
 			{
 				JunkFilter *j;
 
+				resultRelInfo = mtstate->resultRelInfos[i];
 				subplan = mtstate->mt_plans[i]->plan;
 
 				if (operation == CMD_INSERT || operation == CMD_UPDATE)
@@ -2867,13 +2956,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 				}
 
 				resultRelInfo->ri_junkFilter = j;
-				resultRelInfo++;
 			}
 		}
 		else
 		{
 			if (operation == CMD_INSERT)
-				ExecCheckPlanOutput(mtstate->resultRelInfo->ri_RelationDesc,
+				ExecCheckPlanOutput(mtstate->resultRelInfos[0]->ri_RelationDesc,
 									subplan->targetlist);
 		}
 	}
@@ -2920,7 +3008,7 @@ ExecEndModifyTable(ModifyTableState *node)
 	 */
 	for (i = 0; i < node->mt_nplans; i++)
 	{
-		ResultRelInfo *resultRelInfo = node->resultRelInfo + i;
+		ResultRelInfo *resultRelInfo = node->resultRelInfos[i];
 
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
 			resultRelInfo->ri_FdwRoutine != NULL &&
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 739a023965..87339d5e79 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -225,6 +225,7 @@ _copyModifyTable(const ModifyTable *from)
 	COPY_NODE_FIELD(exclRelTlist);
 	COPY_NODE_FIELD(mergeSourceTargetList);
 	COPY_NODE_FIELD(mergeActionList);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e31b6a9c33..b748a1d204 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -393,6 +393,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 	WRITE_NODE_FIELD(mergeSourceTargetList);
 	WRITE_NODE_FIELD(mergeActionList);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 5bf3d28c51..85d7f38d72 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1640,6 +1640,7 @@ _readModifyTable(void)
 	READ_NODE_FIELD(exclRelTlist);
 	READ_NODE_FIELD(mergeSourceTargetList);
 	READ_NODE_FIELD(mergeActionList);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d6c94846d3..c31bb7ea64 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -295,7 +295,8 @@ static ModifyTable *make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam);
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos);
 static GatherMerge *create_gather_merge_plan(PlannerInfo *root,
 						 GatherMergePath *best_path);
 
@@ -2484,6 +2485,7 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 	List	   *subplans = NIL;
 	ListCell   *subpaths,
 			   *subroots;
+	List	   *partpruneinfos = NIL;
 
 	/* Build the plan for each input path */
 	forboth(subpaths, best_path->subpaths,
@@ -2512,6 +2514,27 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (best_path->partitioned_rels != NIL)
+	{
+		int			partrelid = linitial_int(best_path->partitioned_rels);
+		RelOptInfo *rel = root->simple_rel_array[partrelid];
+		List	   *prunequal = NIL;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+											best_path->partitioned_rels,
+											best_path->resultRelations,
+											best_path->subpaths, prunequal);
+	}
+
 	plan = make_modifytable(root,
 							best_path->operation,
 							best_path->canSetTag,
@@ -2527,7 +2550,8 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 							best_path->onconflict,
 							best_path->mergeSourceTargetList,
 							best_path->mergeActionList,
-							best_path->epqParam);
+							best_path->epqParam,
+							partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, &best_path->path);
 
@@ -6600,7 +6624,8 @@ make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam)
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos)
 {
 	ModifyTable *node = makeNode(ModifyTable);
 	List	   *fdw_private_list;
@@ -6662,6 +6687,7 @@ make_modifytable(PlannerInfo *root,
 	node->mergeSourceTargetList = mergeSourceTargetList;
 	node->mergeActionList = mergeActionList;
 	node->epqParam = epqParam;
+	node->part_prune_infos = partpruneinfos;
 
 	/*
 	 * For each result relation that is a foreign table, allow the FDW to
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 421dc79cc4..74fd2db4f7 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1197,6 +1197,7 @@ inheritance_planner(PlannerInfo *root)
 	Query	   *parent_parse;
 	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
 	PlannerInfo **parent_roots = NULL;
+	PlannerInfo *partition_root = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1274,6 +1275,32 @@ inheritance_planner(PlannerInfo *root)
 		 * the ModifyTable node, if one is needed at all.
 		 */
 		partitioned_relids = bms_make_singleton(top_parentRTindex);
+
+		/*
+		 * For partitioned tables, since we're able to determine the minimum
+		 * set of partitions required much more easily than what we can do
+		 * with an inheritance hierarchy, we invoke the grouping_planner on
+		 * the entire given query in order to determine the minimum set of
+		 * partitions which will be required below.  This may mean that we
+		 * invoke the grouping planner far fewer times, as otherwise we'd
+		 * have to invoke it once for each partition.
+		 */
+
+		/*
+		 * Since the planner tends to scribble on the parse, we must make a
+		 * copy of it.  We also must make copies of the PlannerInfo and
+		 * PlannerGlobal since these will also be modified from the call to
+		 * grouping_planner.
+		 */
+		partition_root = makeNode(PlannerInfo);
+		partition_root->glob = makeNode(PlannerGlobal);
+
+		memcpy(partition_root, root, sizeof(PlannerInfo));
+		memcpy(partition_root->glob, root->glob, sizeof(PlannerGlobal));
+
+		partition_root->parse = copyObject(partition_root->parse);
+
+		grouping_planner(partition_root, true, 0.0 /* retrieve all tuples */ );
 	}
 
 	/*
@@ -1304,6 +1331,21 @@ inheritance_planner(PlannerInfo *root)
 		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
+		/*
+		 * If the target rel is a partitioned table then skip any child
+		 * partitions which were found to be dummies by the grouping_planner
+		 * call performed above.
+		 */
+		if (partition_root)
+		{
+			RelOptInfo *rel;
+
+			rel = find_base_rel(partition_root, appinfo->child_relid);
+
+			if (IS_DUMMY_REL(rel))
+				continue;
+		}
+
 		/*
 		 * expand_inherited_rtentry() always processes a parent before any of
 		 * that parent's children, so the parent_root for this relation should
@@ -1629,6 +1671,23 @@ inheritance_planner(PlannerInfo *root)
 		Assert(list_length(partitioned_rels) >= 1);
 	}
 
+	/*
+	 * The individual grouping_planner calls per partition above performed
+	 * no planning on the actual partitioned tables, however, in order to
+	 * allow partition pruning at run-time we must know the baserestrictinfo
+	 * of each partition.  We simply replace the RelOptInfos from the initial
+	 * full plan which was generated and replace the non-complete RelOptInfos
+	 * which are stored in root.
+	 */
+	if (partition_root)
+	{
+		int		i;
+
+		i = -1;
+		while ((i = bms_next_member(partitioned_relids, i)) >= 0)
+			root->simple_rel_array[i] = partition_root->simple_rel_array[i];
+	}
+
 	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
 	add_path(final_rel, (Path *)
 			 create_modifytable_path(root, final_rel,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9851f35f77..6bbf4a8cfa 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1027,6 +1027,8 @@ typedef struct EPQState
 } EPQState;
 
 
+struct PartitionPruning;
+
 /* ----------------
  *	 ResultState information
  * ----------------
@@ -1083,7 +1085,7 @@ typedef struct ModifyTableState
 	PlanState **mt_plans;		/* subplans (one per target rel) */
 	int			mt_nplans;		/* number of plans in the array */
 	int			mt_whichplan;	/* which one is being executed (0..n-1) */
-	ResultRelInfo *resultRelInfo;	/* per-subplan target relations */
+	ResultRelInfo **resultRelInfos;	/* per-subplan target relations */
 	ResultRelInfo *rootResultRelInfo;	/* root target relation (partitioned
 										 * table root) */
 	List	  **mt_arowmarks;	/* per-subplan ExecAuxRowMark lists */
@@ -1109,6 +1111,12 @@ typedef struct ModifyTableState
 
 	/* Flags showing which subcommands are present INS/UPD/DEL/DO NOTHING */
 	int			mt_merge_subcommands;
+
+	/*
+	 * Details required to allow partitions to be eliminated from the scan, or
+	 * NULL if not possible.
+	 */
+	struct PartitionPruning *partition_pruning;
 } ModifyTableState;
 
 /* ----------------
@@ -1130,7 +1138,6 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
-struct PartitionPruning;
 
 struct AppendState
 {
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0b5189aa7d..bfebd48635 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -242,6 +242,8 @@ typedef struct ModifyTable
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 	List	   *mergeSourceTargetList;
 	List	   *mergeActionList;	/* actions for MERGE */
+	List	   *part_prune_infos; /* Mapping details for run-time subplan
+								   * pruning, one per partitioned_rels */
 } ModifyTable;
 
 /* ----------------
-- 
2.16.2.windows.1

#108Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#95)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/03/31 22:52, David Rowley wrote:

The attached patchset is based on Amit's v45 faster partition pruning [1].

I've made a few changes since the v14 version. Since Amit's v45 patch
now creates the partition pruning details in a data structure that can
be copied from the planner over to the executor, it means this patch
is now able to do much of the setup work for run-time pruning in the
planner. Doing this now allows us to determine if run-time pruning is
even possible at plan time, rather than the executor attempting it and
sometimes wasting effort when it failed to find Params matching the
partition key.

Another change from the last version is that I've separated out the
handling of exec Params and external Params. The new patch now will
perform a pruning step at executor startup if some external params
match the partition key. This is very beneficial to a PREPAREd OLTP
type query against a partitioned table as it means we can skip
sub-plan initialisation for all non-matching partitions.

This is quite nice.

Initialising
Append/MergeAppend/ModifyTable nodes with fewer subnodes than were
originally planned did require a small change in explain.c in some
code that was assuming the subnode arrays were the same length as the
subplan list. I also ended up adding a Int property to EXPLAIN to show
the number of subnodes that have been removed during execution.
Unfortunately, there is a small corner case with this in the case
where all subnodes are removed it leaves EXPLAIN with no subnodes to
search for outer side Vars in. I didn't really see any sane way to
handle this in EXPLAIN, so I think the best fix for this is what I've
done, and that's just to always initalise at least 1 subnode, even
when none match the external Params. Cost-wise, I don't think this is
such a big deal as the cost savings here are coming from saving on
initalising ten's or hundreds of subnodes, not 1.

I have wondered about the possibility of setting a valid (non-dummy)
targetlist in Append and MergeAppend if they're created for a partitioned
table. Currently they're overwritten by a dummy one using
set_dummy_tlist_references() in set_plan_refs() citing the following reason:

* set_dummy_tlist_references
* Replace the targetlist of an upper-level plan node with a simple
* list of OUTER_VAR references to its child.
*
* This is used for plan types like Sort and Append that don't evaluate
* their targetlists. Although the executor doesn't care at all what's in
* the tlist, EXPLAIN needs it to be realistic.

In fact, when I had noticed that this EXPLAIN behavior I had wondered if
that's something we should have discussed when d3cc37f1d801a [1]https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=d3cc37f1d801a went in.

To put the new patch to the test, I tried pgbench -S -M prepared -s
100 with and without having modified pgbench_accounts to separate into
10 RANGE partitions of equal size.

A non-partitioned table was getting 12503 TPS.
With partitioned tables, the old version of this patch was getting: 5470 TPS.
With partitioned tables, the attached version gets 11247 TPS.
For perspective, today's master with a partitioned table gets 4719 TPS.

Quite nice!

So you can see it's a pretty good performance boost by skipping
initialisation of the 9 non-matching subplans. It's not hard to
imagine the gains getting more significant with a larger number of
partitions. Ideally, the performance of a partitioned table would be
faster than the non-partitioned case, but please remember this query
is a single row PK lookup SELECT, so is a very fast query in both
cases. Partitioning cases should improve more as the table grows and
indexes struggle to fit in RAM, or when many rows are being taken from
the partition and being aggregated.

Unfortunately, the story is not quite as good for the non -M prepared
version of the benchmark. This shows that planning time of partitioned
table queries could still use some improvements. Amit's v45 patch
certainly makes a dent in the planner slow performance here, but it's
still nothing like as fast as the non-partitioned case. More work is
required there in the future.

Certainly. Things like the issue that we both replied to yesterday should
be addressed for starters [2]/messages/by-id/20180403194613.GY28454@telsasoft.com.

This patchset also contains a patch to improve the performance of
inheritance planning of UPDATE/DELETE queries. This patch also
implements run-time pruning for UPDATE/DELETE too. This also has a
significant performance improvement for planning of UPDATE/DELETE
operations on partitioned tables with a large number of partitions,
performance is as follows:

Values in transactions per second.

Partitions = 1
Unpatched: 7323.3
Patched: 6573.2 (-10.24%)

Partitions = 2
Unpatched: 6784.8
Patched: 6377.1 (-6.01%)

Partitions = 4
Unpatched: 5903.0
Patched: 6106.8 (3.45%)

Partitions = 8
Unpatched: 4582.0
Patched: 5579.9 (21.78%)

Partitions = 16
Unpatched: 3131.5
Patched: 4521.2 (44.38%)

Partitions = 32
Unpatched: 1779.8
Patched: 3387.8 (90.35%)

Partitions = 64
Unpatched: 821.9
Patched: 2245.4 (173.18%)

Partitions = 128
Unpatched: 322.2
Patched: 1319.6 (309.56%)

Partitions = 256
Unpatched: 84.3
Patched: 731.7 (768.27%)

Partitions = 512
Unpatched: 22.5
Patched: 382.8 (1597.74%)

Partitions = 1024
Unpatched: 5.5
Patched: 150.1 (2607.83%)

Great!

I will post comments on your v19 later today.

[1]: https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=d3cc37f1d801a
https://git.postgresql.org/gitweb/?p=postgresql.git;a=commit;h=d3cc37f1d801a

[2]: /messages/by-id/20180403194613.GY28454@telsasoft.com
/messages/by-id/20180403194613.GY28454@telsasoft.com

#109David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#108)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 5 April 2018 at 15:14, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

On 2018/03/31 22:52, David Rowley wrote:

Initialising
Append/MergeAppend/ModifyTable nodes with fewer subnodes than were
originally planned did require a small change in explain.c in some
code that was assuming the subnode arrays were the same length as the
subplan list. I also ended up adding a Int property to EXPLAIN to show
the number of subnodes that have been removed during execution.
Unfortunately, there is a small corner case with this in the case
where all subnodes are removed it leaves EXPLAIN with no subnodes to
search for outer side Vars in. I didn't really see any sane way to
handle this in EXPLAIN, so I think the best fix for this is what I've
done, and that's just to always initalise at least 1 subnode, even
when none match the external Params. Cost-wise, I don't think this is
such a big deal as the cost savings here are coming from saving on
initalising ten's or hundreds of subnodes, not 1.

I have wondered about the possibility of setting a valid (non-dummy)
targetlist in Append and MergeAppend if they're created for a partitioned
table. Currently they're overwritten by a dummy one using
set_dummy_tlist_references() in set_plan_refs() citing the following reason:

* set_dummy_tlist_references
* Replace the targetlist of an upper-level plan node with a simple
* list of OUTER_VAR references to its child.
*
* This is used for plan types like Sort and Append that don't evaluate
* their targetlists. Although the executor doesn't care at all what's in
* the tlist, EXPLAIN needs it to be realistic.

In fact, when I had noticed that this EXPLAIN behavior I had wondered if
that's something we should have discussed when d3cc37f1d801a [1] went in.

I had a quick hack at this to see if it would work and it does seem to
on my very simple test. However, it would mean removing
set_dummy_tlist_references from more than just Append/MergeAppend

create table listp (a int, b int) partition by list(a);
create table listp1 partition of listp for values in(1);
create table listp2 partition of listp for values in(2);
prepare q1 (int, int) as select * from listp where a in($1,$2) order
by b limit 1;
explain execute q1(1,2);
explain execute q1(1,2);
explain execute q1(1,2);
explain execute q1(1,2);
explain execute q1(1,2);
explain (verbose, costs off) execute q1(0,0);
QUERY PLAN
--------------------------------------------------------
Limit
Output: listp.a, listp.b
-> Sort
Output: listp.a, listp.b
Sort Key: listp.b
-> Append
Subplans Pruned: 2
(7 rows)

The downside is that if we were to do this it would mean changing the
output in cases like:

explain (verbose, costs off) (select a z, b y from listp union all
select * from listp) order by y;
QUERY PLAN
--------------------------------------------------------------------------------------
Sort
Output: z, y
Sort Key: y
-> Append
-> Seq Scan on public.listp1
Output: listp1.a, listp1.b
-> Seq Scan on public.listp2
Output: listp2.a, listp2.b
-> Seq Scan on public.listp1 listp1_1
Output: listp1_1.a, listp1_1.b
-> Seq Scan on public.listp2 listp2_1
Output: listp2_1.a, listp2_1.b

Notice the sort key now refers to the alias rather than a column from
the first append child.

It sure is an interesting thought, and one I'd not considered, but I
don't think trying for something like this is going to be the path of
least resistance. It may also add quite a bit of complexity if we just
try to do it when the OUTER_VAR would lead to a Append/MergeAppend
which belongs to a partitioned table scan.

I think this idea has good merit and some form of it might be the
nicest way to allow run-time pruning of all subnodes in the future.
Perhaps we can put it on the shelf and think about it again for PG12.
However, it might not be the most interesting optimization to work on,
as I think probably run-time pruning away of all subnodes is probably
far less common than pruning some, or all but one, and the cost of
initializing the one unneeded subnode is not so big.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

set_dummy_tlist_hacks.patchapplication/octet-stream; name=set_dummy_tlist_hacks.patchDownload
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 5286ada835..a25e83d606 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -152,13 +152,13 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 			 * first subplan, but we set a noop flag so that we never actually
 			 * bother scanning it.
 			 */
-			if (bms_is_empty(validsubplans))
-			{
-				appendstate->as_noopscan = true;
+			//if (bms_is_empty(validsubplans))
+			//{
+			//	appendstate->as_noopscan = true;
 
-				/* Mark the first as valid so that it's initialized below */
-				validsubplans = bms_make_singleton(0);
-			}
+			//	/* Mark the first as valid so that it's initialized below */
+			//	validsubplans = bms_make_singleton(0);
+			//}
 
 			nplans = bms_num_members(validsubplans);
 		}
@@ -173,7 +173,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		 * If there's no exec params then no further pruning can be done, we
 		 * can just set the valid subplans to all remaining subplans.
 		 */
-		if (bms_is_empty(partprune->execparams))
+		if (bms_is_empty(partprune->execparams) && nplans > 0)
 			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
 
 		appendstate->partition_pruning = partprune;
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index cd540a0df5..29873cd0df 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -652,7 +652,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 			 * executor, we fix it up for possible use by EXPLAIN (not to
 			 * mention ease of debugging --- wrong varnos are very confusing).
 			 */
-			set_dummy_tlist_references(plan, rtoffset);
+			//set_dummy_tlist_references(plan, rtoffset);
 
 			/*
 			 * Since these plan types don't check quals either, we should not
@@ -669,7 +669,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 				 * tlist or quals.  But we have to fix up the RT indexes in
 				 * its rowmarks.
 				 */
-				set_dummy_tlist_references(plan, rtoffset);
+				//set_dummy_tlist_references(plan, rtoffset);
 				Assert(splan->plan.qual == NIL);
 
 				foreach(l, splan->rowMarks)
@@ -691,7 +691,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 				 * however; and those cannot contain subplan variable refs, so
 				 * fix_scan_expr works for them.
 				 */
-				set_dummy_tlist_references(plan, rtoffset);
+				//set_dummy_tlist_references(plan, rtoffset);
 				Assert(splan->plan.qual == NIL);
 
 				splan->limitOffset =
@@ -971,7 +971,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 				 * Append, like Sort et al, doesn't actually evaluate its
 				 * targetlist or check quals.
 				 */
-				set_dummy_tlist_references(plan, rtoffset);
+				//set_dummy_tlist_references(plan, rtoffset);
 				Assert(splan->plan.qual == NIL);
 				foreach(l, splan->partitioned_rels)
 				{
@@ -993,7 +993,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 				 * MergeAppend, like Sort et al, doesn't actually evaluate its
 				 * targetlist or check quals.
 				 */
-				set_dummy_tlist_references(plan, rtoffset);
+				//set_dummy_tlist_references(plan, rtoffset);
 				Assert(splan->plan.qual == NIL);
 				foreach(l, splan->partitioned_rels)
 				{
@@ -1009,7 +1009,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
 			break;
 		case T_RecursiveUnion:
 			/* This doesn't evaluate targetlist or check quals either */
-			set_dummy_tlist_references(plan, rtoffset);
+			//set_dummy_tlist_references(plan, rtoffset);
 			Assert(plan->qual == NIL);
 			break;
 		case T_BitmapAnd:
#110David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#107)
5 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 5 April 2018 at 14:01, David Rowley <david.rowley@2ndquadrant.com> wrote:

I've attached an updated patchset against Amit's v49 faster partition
pruning patch.

Well, v19 was short lived. 0005 conflicted with some recent changes to MERGE.

The attached v20 patch set is now based on Amit's runtime partition
prune v50 [1]/messages/by-id/77a518ac-e4a0-4cd1-9988-e5d754a6501f@lab.ntt.co.jp.

There are a few changes since v19:

1. I've run pgindent on the entire patchset.
2. Made a pass over the comments and fixed a few things that were not
quite right. Also improved some wording.
3. Fixed a small error in make_partition_pruneinfo where I had
accidentally used the 'i' variable for two purposes at once. There was
no live bug there, but I've now changed things around as what was
there was wrong regardless of if it was causing issues or not.
4. updated typedefs.list with new typedefs added in the patchset.

[1]: /messages/by-id/77a518ac-e4a0-4cd1-9988-e5d754a6501f@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v20-0001-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v20-0001-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From fea470663c9ee8233ab47a208a715e821d1d7edf Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 00:35:00 +1200
Subject: [PATCH v20 1/5] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/catalog/partition.c        |  24 ++
 src/backend/commands/explain.c         |  51 ++--
 src/backend/executor/execPartition.c   | 478 +++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c          |  20 ++
 src/backend/nodes/outfuncs.c           |  27 ++
 src/backend/nodes/readfuncs.c          |  19 ++
 src/backend/optimizer/util/partprune.c | 261 ++++++++++++++++++
 src/include/catalog/partition.h        |  13 +
 src/include/executor/execPartition.h   |  80 ++++++
 src/include/nodes/nodes.h              |   1 +
 src/include/nodes/primnodes.h          |  23 ++
 src/include/optimizer/partprune.h      |   4 +
 src/tools/pgindent/typedefs.list       |   3 +
 13 files changed, 986 insertions(+), 18 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 73631ca0e7..7fa20abd33 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1937,6 +1937,30 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+
+			/*
+			 * 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;
+				bool		isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+												   context->planstate->ps_ExprContext,
+												   &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+
+			}
 		default:
 			break;
 	}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 79f639d5e2..549622da93 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1811,28 +1811,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3173,18 +3178,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that was
+	 * specified in the plan means that some subnodes have been ignored per
+	 * instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index ad532773a3..800b6acbd4 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,6 +39,12 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_extparams_recurse(
+									PartitionRelPruning * partrelprune,
+									Bitmapset **validsubplans);
+static void find_subplans_for_allparams_recurse(
+									PartitionRelPruning * partrelprune,
+									Bitmapset **validsubplans);
 
 
 /*
@@ -1272,3 +1278,475 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all.  This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruning:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruning details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruning
+ *		Setup the required data structure which is required for calling
+ *		ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These contexts can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionRelPruning *partrelprunes;
+	PartitionPruning *partprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	partprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionRelPruning *)
+		palloc(sizeof(PartitionRelPruning) *
+			   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	partprune->partrelpruning = partrelprunes;
+	partprune->npartrelpruning = list_length(partitionpruneinfo);
+	partprune->extparams = NULL;
+	partprune->execparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	partprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+													 "Partition Prune",
+													 ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc partdesc;
+		Relation	rel;
+		PartitionKey partkey;
+		int			partnatts;
+		int			j;
+
+		partrelprune->allpartindexes = bms_copy(pinfo->allpartindexes);
+		partrelprune->nparts = pinfo->nparts;
+		partrelprune->subnodeindex = palloc(sizeof(int) * pinfo->nparts);
+		partrelprune->subpartprune = palloc(sizeof(PartitionRelPruning *) *
+											pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(partrelprune->subnodeindex, pinfo->subnodeindex,
+			   sizeof(int) * pinfo->nparts);
+
+		/*
+		 * Setup the PartitionedRelPruning's subpartprune so that we can
+		 * quickly find sub-PartitionedRelPruning details for any
+		 * sub-partitioned tables that this partitioned table contains.
+		 * We need to be able to find these quickly during our recursive
+		 * search to find all matching subnodes.
+		 */
+		for (j = 0; j < pinfo->nparts; j++)
+		{
+			int			subpartidx = pinfo->subpartindex[j];
+
+			Assert(subpartidx < list_length(partitionpruneinfo));
+
+			if (subpartidx >= 0)
+				partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
+			else
+				partrelprune->subpartprune[j] = NULL;
+		}
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL; /* empty for now */
+
+		partrelprune->prunesteps = pinfo->prunesteps;
+
+		partrelprune->extparams = bms_copy(pinfo->extparams);
+		partrelprune->allparams = bms_union(pinfo->extparams,
+											pinfo->execparams);
+
+		/*
+		 * Accumulate the paramids which match the partitioned keys of all
+		 * partitioned tables.
+		 */
+		partprune->extparams = bms_add_members(partprune->extparams,
+											   pinfo->extparams);
+
+		partprune->execparams = bms_add_members(partprune->execparams,
+												pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the paramids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	partprune->allparams = bms_union(partprune->extparams,
+									 partprune->execparams);
+
+	return partprune;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'partprune'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so only once during their
+ * init plan.  The caller must only initialize the subnodes which are returned
+ * by this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'partprune' has any extparams.
+ *
+ * 'nsubplans' must be passed as the total number of unpruned subplans.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(partprune->extparams));
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	/* Determine which subplans match these external params */
+	find_subplans_for_extparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been performed for external Params.
+	 */
+	bms_free(partprune->extparams);
+	partprune->extparams = NULL;
+
+	/*
+	 * If any subplans were pruned, we must re-sequence the subplan indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subplans which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubplans)
+	{
+		int		   *subplanidxmap;
+		int			i;
+		int			newidx;
+
+		/*
+		 * First we must build a map which allows us to map the old subplan
+		 * index into the new one.
+		 */
+		subplanidxmap = (int *) palloc(sizeof(int) * nsubplans);
+		newidx = 0;
+		for (i = 0; i < nsubplans; i++)
+		{
+			if (bms_is_member(i, result))
+				subplanidxmap[i] = newidx++;
+			else
+				subplanidxmap[i] = -1;	/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnodeindex so
+		 * that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < partprune->npartrelpruning; i++)
+		{
+			PartitionRelPruning *partrelprune;
+			int			j;
+
+			partrelprune = &partprune->partrelpruning[i];
+
+			/*
+			 * We also need to reset the allpartindexes field so that it only
+			 * contains partition indexes that we actually still have
+			 * subnodeindexes for.  It seems easier to build a fresh one,
+			 * rather than trying to update the existing one.
+			 */
+			bms_free(partrelprune->allpartindexes);
+			partrelprune->allpartindexes = NULL;
+
+			for (j = 0; j < partrelprune->nparts; j++)
+			{
+				int			oldidx = partrelprune->subnodeindex[j];
+
+				/*
+				 * If this partition existed as a subplan then change the old
+				 * subplan index to the new subplan index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subplan list due to some subplans
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					partrelprune->subnodeindex[j] = subplanidxmap[oldidx];
+
+					if (subplanidxmap[oldidx] >= 0)
+						partrelprune->allpartindexes =
+							bms_add_member(partrelprune->allpartindexes,
+										   j);
+				}
+			}
+		}
+
+		pfree(subplanidxmap);
+	}
+
+
+	return result;
+}
+
+/*
+ * find_subplans_for_extparams_recurse
+ *		Recursive worker function for ExecFindInitialMatchingSubPlans.
+ */
+static void
+find_subplans_for_extparams_recurse(PartitionRelPruning * partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset  *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * We only need to determine the matching partitions if there are any ext
+	 * params matching the partition key at this level.  If there are no such
+	 * 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.
+	 */
+	if (!bms_is_empty(partrelprune->extparams))
+	{
+		context->safeparams = partrelprune->extparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_extparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'partprune' for the current Param values.
+ *
+ * Here we utilize both external and exec Params for pruning.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *partprune)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	partrelprune = partprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+
+	find_subplans_for_allparams_recurse(partrelprune, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(partprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_allparams_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans.
+ */
+static void
+find_subplans_for_allparams_recurse(PartitionRelPruning * partrelprune,
+									Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset  *partset;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(partrelprune->allparams))
+	{
+		context->safeparams = partrelprune->allparams;
+		partset = get_matching_partitions(context, partrelprune->prunesteps);
+	}
+	else
+		partset = partrelprune->allpartindexes;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnodeindex[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnodeindex[i]);
+		else if (partrelprune->subpartprune[i] != NULL)
+			find_subplans_for_allparams_recurse(partrelprune->subpartprune[i],
+												validsubplans);
+		else
+		{
+			/*
+			 * If this happens then we're somehow missing a subnode.  This
+			 * shouldn't happen and could only happen if a more restrictive
+			 * clause list was used for partition elimination during planning
+			 * than what was used here.
+			 */
+			elog(ERROR, "partition missing from subplans");
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b0fa556f71..96501b5ba5 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2167,6 +2167,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(prunesteps);
+	COPY_BITMAPSET_FIELD(allpartindexes);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5103,6 +5120,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e6793b4716..ea7c4b7ac6 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1746,6 +1746,30 @@ _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(prunesteps);
+	WRITE_BITMAPSET_FIELD(allpartindexes);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnodeindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnodeindex[i]);
+
+	appendStringInfoString(str, " :subpartindex");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpartindex[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3977,6 +4001,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 25874074a0..b3a4f45da4 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1357,6 +1357,23 @@ _readPartitionPruneStepCombine(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(prunesteps);
+	READ_BITMAPSET_FIELD(allpartindexes);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnodeindex, local_node->nparts);
+	READ_INT_ARRAY(subpartindex, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2626,6 +2643,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 5b306193e1..a9fe4bf2ed 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -27,6 +27,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -81,6 +82,7 @@ typedef struct GeneratePruningStepsContext
 	List   *steps;
 }			GeneratePruningStepsContext;
 
+static bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static List *generate_partition_pruning_steps_internal(RelOptInfo *rel,
 									  GeneratePruningStepsContext *context,
 									  List *clauses,
@@ -166,6 +168,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 		context.nparts = rel->nparts;
 		context.boundinfo = rel->boundinfo;
 
+		/* Not valid when being called from the planner */
+		context.planstate = NULL;
+		context.safeparams = NULL;
+
 		partindexes = get_matching_partitions(&context, pruning_steps);
 
 		/* Add selected partitions' RT indexes to result. */
@@ -230,8 +236,263 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 	return context.steps;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Builds List of PartitionPruneInfos, one for each 'partitioned_rels'.
+ *		These can be used in the executor to allow additional partition
+ *		pruning to take place.
+ *
+ * Here we generate partition pruning steps for 'prunequal' and also build a
+ * data stucture which allows mapping of partition indexes into 'subpaths'
+ * or 'resultRelations' indexes.
+ *
+ * If 'resultRelations' is non-NIL, then this List of relids is used to build
+ * the mapping structures.  Otherwise the 'subpaths' List is used.
+ *
+ * If no Params were found to match the partition key in any of the
+ * 'partitioned_rels', then we return NIL.  In such a case run-time partition
+ * pruning would be useless.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal)
+{
+	RangeTblEntry *rte;
+	RelOptInfo *parentpart = NULL;
+	ListCell   *lc;
+	List	   *pinfolist = NIL;
+	int		   *allsubnodeindex;
+	int		   *allsubpartindex;
+	int			i;
+	bool		gotparam = false;
+
+	/*
+	 * Allocate two arrays, one to allow quick lookups of the 'subpaths' index
+	 * of a relation by relid and another to lookup the 'partitioned_rel'
+	 * index by relid.
+	 */
+	allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
+	allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);
+
+	/* Initialize to -1 to indicate the rel was not found */
+	for (i = 0; i < root->simple_rel_array_size; i++)
+	{
+		allsubnodeindex[i] = -1;
+		allsubpartindex[i] = -1;
+	}
+
+	/*
+	 * If 'resultRelations' are present then map these, otherwise we map
+	 * the 'subpaths' List.
+	 */
+	if (resultRelations != NIL)
+	{
+		i = 0;
+		foreach(lc, resultRelations)
+		{
+			int			resultrel = lfirst_int(lc);
+
+			Assert(resultrel < root->simple_rel_array_size);
+			allsubnodeindex[resultrel] = i++;
+		}
+	}
+	else
+	{
+		i = 0;
+		foreach(lc, subpaths)
+		{
+			Path	   *path = (Path *) lfirst(lc);
+			RelOptInfo *pathrel = path->parent;
+
+			Assert(IS_SIMPLE_REL(pathrel));
+			Assert(pathrel->relid < root->simple_rel_array_size);
+
+			allsubnodeindex[pathrel->relid] = i++;
+		}
+	}
+
+	/* Likewise for the partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		allsubpartindex[rti] = i;
+		i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	i = 0;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		int			nparts = subpart->nparts;
+		int		   *subnodeindex;
+		int		   *subpartindex;
+		List	   *partprunequal;
+		bool		constfalse;
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (!parentpart)
+		{
+			parentpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+				adjust_appendrel_attrs_multilevel(root,
+												  (Node *) prunequal,
+												  subpart->relids,
+												  parentpart->relids);
+		}
+
+		pinfo->prunesteps = generate_partition_pruning_steps(subpart,
+															 partprunequal,
+															 &constfalse);
+
+		if (constfalse)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered.  We'd
+			 * better do something sane here anyway, so let's just disable
+			 * run-time pruning.
+			 */
+			return NIL;
+		}
+
+		pinfo->allpartindexes = NULL;
+		pinfo->nparts = nparts;
+		pinfo->subnodeindex = subnodeindex = palloc(nparts * sizeof(int));
+		pinfo->subpartindex = subpartindex = palloc(nparts * sizeof(int));
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+
+		/*
+		 * 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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pinfo->prunesteps);
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present in
+		 * the subpaths List will be set to -1, and any subpartitioned table
+		 * which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = allsubnodeindex[partrel->relid];
+			int			subpartidx = allsubpartindex[partrel->relid];
+
+			subnodeindex[i] = subnodeidx;
+			subpartindex[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				pinfo->allpartindexes = bms_add_member(pinfo->allpartindexes,
+													   i);
+		}
+
+		pinfolist = lappend(pinfolist, pinfo);
+	}
+
+	pfree(allsubnodeindex);
+	pfree(allsubpartindex);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
+
 /* Module-local functions */
 
+/*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell   *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							 (int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
 /*
  * generate_partition_pruning_steps_internal
  *		Processes 'clauses' to generate partition pruning steps.
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index b4b4844f20..acacbe5127 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -113,6 +114,18 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow
+	 * params found matching the partition key to be evaulated.
+	 */
+	PlanState  *planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until after init plan.
+	 */
+	Bitmapset  *safeparams;
 } PartitionPruneContext;
 
 extern void RelationBuildPartitionDesc(Relation relation);
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 9f55f6409e..b2fcf894ad 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "optimizer/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,80 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows partition.c's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'prunesteps' will be found within.
+ *
+ * nparts						The number of partitions which belong to this
+ *								partitioned relation. Also defines the size of
+ *								the 'subnodeindex' and 'subpartprune' arrays.
+ * subnodeindex					An array of nparts containing the subnode
+ *								index which matches this partition index, or
+ *								-1 if there is no match.
+ * subpartprune					An array of nparts containing the
+ *								PartitionRelPruning details this partition
+ *								index for sub-partitioned tables.
+ * allpartindexes				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ *								belong to this partition.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * prunesteps					Contains list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ * extparams					Contains paramids of external params found
+ *								matching partition keys in 'prunesteps'.
+ * allparams					As 'extparams' but also including exec params.
+ *-----------------------
+ */
+typedef struct PartitionRelPruning
+{
+	int			nparts;
+	int		   *subnodeindex;
+	struct PartitionRelPruning **subpartprune;
+	Bitmapset  *allpartindexes;
+	PartitionPruneContext context;
+	List	   *prunesteps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionRelPruning
+ * structs and also stores all paramids which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partrelpruning		Array of PartitionRelPruning for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * npartrelpruning		Number of items in partrelpruning array.
+ * 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
+ *						partition key in each of the contained
+ *						PartitionRelPruning structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of extparams and execparams, saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionRelPruning *partrelpruning;
+	int			npartrelpruning;
+	MemoryContext prune_context;
+	Bitmapset  *extparams;
+	Bitmapset  *execparams;
+	Bitmapset  *allparams;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -127,5 +202,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+								int nsubplans);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 1ec8030d4b..940469e41a 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -196,6 +196,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 965eb656a8..53148a0e55 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1579,4 +1579,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;			/* Oid of partition rel */
+	List	   *prunesteps;		/* List of PartitionPruneStep */
+	Bitmapset  *allpartindexes; /* All part index we have subnodes for at this
+								 * level */
+	int			nparts;			/* length of the following arrays */
+	int		   *subnodeindex;	/* subnode index indexed by partition id */
+	int		   *subpartindex;	/* subpart index indexed by partition id */
+	Bitmapset  *extparams;		/* All external ParamIDs seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec ParamIDs seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 1f2fe297a3..b7352d150c 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -20,4 +20,8 @@ extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *resultRelations, List *subpaths,
+						 List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index adde8eaee9..76ea85b733 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1602,12 +1602,15 @@ PartitionHashBound
 PartitionKey
 PartitionListValue
 PartitionPruneContext
+PartitionPruneInfo
 PartitionPruneStep
 PartitionPruneStepCombine
 PartitionPruneStepOp
+PartitionPruning
 PartitionRangeBound
 PartitionRangeDatum
 PartitionRangeDatumKind
+PartitionRelPruning
 PartitionScheme
 PartitionSpec
 PartitionTupleRouting
-- 
2.16.2.windows.1

v20-0002-Add-bms_prev_member-function.patchapplication/octet-stream; name=v20-0002-Add-bms_prev_member-function.patchDownload
From dd42534a2385a0317fda282d4fe6a58df67517e1 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 00:35:26 +1200
Subject: [PATCH v20 2/5] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1088,6 +1110,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 	return -2;
 }
 
+/*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
 /*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.16.2.windows.1

v20-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchapplication/octet-stream; name=v20-0003-Allow-Append-subnodes-to-be-pruned-during-execut.patchDownload
From 7241913614a3990c554d2226f3b2f5ba6df02eb3 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 00:35:56 +1200
Subject: [PATCH v20 3/5] Allow Append subnodes to be pruned during execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  256 +++++-
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   15 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1800 insertions(+), 86 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..67738a4b57 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +101,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -112,55 +116,118 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+	appendstate->as_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&appendstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+															list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set a noop flag so that we never actually
+			 * bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+											 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the appendplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
+
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +318,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +350,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +440,40 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/* Handle the case for when all subplans were pruned */
+	if (node->as_noopscan)
+		return false;
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -411,10 +502,33 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
 	else
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +581,25 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
+
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+			ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +665,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int			i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 96501b5ba5..ee4c70aef8 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -248,6 +248,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index c3f1789ce2..db1470ce16 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3802,32 +3802,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3877,15 +3877,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index ea7c4b7ac6..e22ad4d4dd 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -418,6 +418,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index b3a4f45da4..2e22da6c4f 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1675,6 +1675,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index c36a254ed6..25cfa5e349 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1607,7 +1607,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1649,8 +1649,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1698,7 +1698,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1761,7 +1761,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2027,7 +2027,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99d0736029..7560be9522 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1041,6 +1042,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1078,6 +1081,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels, NIL,
+													  best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1086,7 +1120,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5382,7 +5417,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5394,7 +5430,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 008492bad5..421dc79cc4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3920,7 +3920,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6852,8 +6853,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 67e47887fc..2ce4d4496d 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 416b3f9578..bd9442c22d 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3574,7 +3592,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ff63d179b2..b63c0c5329 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1114,8 +1114,15 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
+ *		noopscan			true if partition pruning proved that none of the
+ *							appendplans can contain a record to satisfy this
+ *							query.
  * ----------------
  */
 
@@ -1123,6 +1130,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruning;
 
 struct AppendState
 {
@@ -1132,7 +1140,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
+	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0a797f0a05..c3e5c2c79f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -256,6 +256,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 895bf6959d..4f65686d9b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 2d77b3edd4..0a2517125c 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1516,3 +1516,1138 @@ explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 (9 rows)
 
 drop table hp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index ad5177715c..eefbf32e93 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -274,3 +274,347 @@ explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and
 explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 
 drop table hp;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.16.2.windows.1

v20-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchapplication/octet-stream; name=v20-0004-Allow-MergeAppend-s-subnodes-to-be-pruned-during.patchDownload
From ab1549156e5231c8d9d303a0570eb6358d159371 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 00:36:33 +1200
Subject: [PATCH v20 4/5] Allow MergeAppend's subnodes to be pruned during
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   5 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 353 insertions(+), 21 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..b44a38f53a 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,10 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -78,19 +81,89 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mergestate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+															list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(partprune->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->partition_pruning = partprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +174,24 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +249,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +376,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ee4c70aef8..739a023965 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -276,6 +276,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e22ad4d4dd..e31b6a9c33 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -450,6 +450,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2e22da6c4f..5bf3d28c51 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1697,6 +1697,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 7560be9522..4e54fe6d25 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1144,6 +1144,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1229,8 +1231,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels, NIL,
+													  best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index b63c0c5329..1986abaa9c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1155,6 +1155,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1167,6 +1173,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c3e5c2c79f..a7dbd31466 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -280,6 +280,11 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 0a2517125c..792924fe0b 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2651,3 +2651,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index eefbf32e93..6f389bed1f 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -618,3 +618,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.16.2.windows.1

v20-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchapplication/octet-stream; name=v20-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchDownload
From 6ba5abb5c4dc5d01a27d2414e715dd689af6db5e Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 00:37:13 +1200
Subject: [PATCH v20 5/5] Improve planning speed of partitioned table
 UPDATE/DELETEs

By making a call to grouping_planner for the complete parse of the query we
can make use of the faster partition pruning code used there.  This will
identify all partitions which could be pruned as IS_DUMMY_RELs, of which we
can skip performing each individual grouping_planner call inside
inheritance_planner.  This can improve planner performance significantly when
there are many partitions. There may be a slight slowdown when no partitions
could be pruned or when there are very few (1 or 2) partitions.  However it
seems better to optimize the case when partitions are pruned, rather than the
case where they're not, as those queries are less likely to be fast to
execute. The case for partitioned tables with just 1 or 2 leaf partitions does
not seem worth worrying about too much. The measured regression on 1 partition
was just 10% of overall planning time.

This commit also implements run-time partition pruning for UPDATE/DELETE.
---
 src/backend/commands/explain.c          |   4 +-
 src/backend/executor/execMerge.c        |   6 +-
 src/backend/executor/execPartition.c    |  22 ++---
 src/backend/executor/nodeModifyTable.c  | 163 ++++++++++++++++++++++++--------
 src/backend/nodes/copyfuncs.c           |   1 +
 src/backend/nodes/outfuncs.c            |   1 +
 src/backend/nodes/readfuncs.c           |   1 +
 src/backend/optimizer/plan/createplan.c |  32 ++++++-
 src/backend/optimizer/plan/planner.c    |  59 ++++++++++++
 src/include/nodes/execnodes.h           |  11 ++-
 src/include/nodes/plannodes.h           |   2 +
 11 files changed, 244 insertions(+), 58 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 549622da93..12c933056d 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -3028,14 +3028,14 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 	/* Should we explicitly label target relations? */
 	labeltargets = (mtstate->mt_nplans > 1 ||
 					(mtstate->mt_nplans == 1 &&
-					 mtstate->resultRelInfo->ri_RangeTableIndex != node->nominalRelation));
+					 mtstate->resultRelInfos[0]->ri_RangeTableIndex != node->nominalRelation));
 
 	if (labeltargets)
 		ExplainOpenGroup("Target Tables", "Target Tables", false, es);
 
 	for (j = 0; j < mtstate->mt_nplans; j++)
 	{
-		ResultRelInfo *resultRelInfo = mtstate->resultRelInfo + j;
+		ResultRelInfo *resultRelInfo = mtstate->resultRelInfos[j];
 		FdwRoutine *fdwroutine = resultRelInfo->ri_FdwRoutine;
 
 		if (labeltargets)
diff --git a/src/backend/executor/execMerge.c b/src/backend/executor/execMerge.c
index d39ddd3034..7ff1bfc96b 100644
--- a/src/backend/executor/execMerge.c
+++ b/src/backend/executor/execMerge.c
@@ -203,8 +203,8 @@ ExecMergeMatched(ModifyTableState *mtstate, EState *estate,
 		if (resultRelInfo == NULL)
 		{
 			resultRelInfo = ExecInitPartitionInfo(mtstate,
-					mtstate->resultRelInfo,
-					proute, estate, leaf_part_index);
+												  mtstate->resultRelInfos[0],
+												  proute, estate, leaf_part_index);
 			Assert(resultRelInfo != NULL);
 		}
 	}
@@ -500,7 +500,7 @@ ExecMergeNotMatched(ModifyTableState *mtstate, EState *estate,
 	 * the currently active result relation, which corresponds to the root
 	 * of the partition tree.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 
 	/*
 	 * For INSERT actions, root relation's merge action is OK since the
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 800b6acbd4..ea8f21de2f 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -103,7 +103,7 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
 
 	if (is_update)
 	{
-		update_rri = mtstate->resultRelInfo;
+		update_rri = mtstate->resultRelInfos[0];
 		num_update_rri = list_length(node->plans);
 		proute->subplan_partition_offsets =
 			palloc(num_update_rri * sizeof(int));
@@ -423,8 +423,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * In the case of INSERT on a partitioned table, there is only one
@@ -479,8 +479,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		TupleTableSlot *slot;
 		ExprContext *econtext;
 		List	   *returningList;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/* See the comment above for WCO lists. */
 		Assert((node->operation == CMD_INSERT &&
@@ -535,8 +535,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	if (node && node->onConflictAction != ONCONFLICT_NONE)
 	{
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		ExprContext *econtext = mtstate->ps.ps_ExprContext;
 		ListCell   *lc;
@@ -676,8 +676,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	{
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * If the root parent and partition have the same tuple
@@ -1612,7 +1612,7 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
  *		Recursive worker function for ExecFindInitialMatchingSubPlans.
  */
 static void
-find_subplans_for_extparams_recurse(PartitionRelPruning * partrelprune,
+find_subplans_for_extparams_recurse(PartitionRelPruning *partrelprune,
 									Bitmapset **validsubplans)
 {
 	PartitionPruneContext *context = &partrelprune->context;
@@ -1701,7 +1701,7 @@ ExecFindMatchingSubPlans(PartitionPruning *partprune)
  *		Recursive worker function for ExecFindMatchingSubPlans.
  */
 static void
-find_subplans_for_allparams_recurse(PartitionRelPruning * partrelprune,
+find_subplans_for_allparams_recurse(PartitionRelPruning *partrelprune,
 									Bitmapset **validsubplans)
 {
 	PartitionPruneContext *context = &partrelprune->context;
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 0ebf37bd24..1513ffcf90 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -1214,12 +1214,12 @@ lreplace:;
 				map_index = resultRelInfo->ri_PartitionLeafIndex;
 				Assert(mtstate->rootResultRelInfo == NULL);
 				tupconv_map = TupConvMapForLeaf(proute,
-								mtstate->resultRelInfo,
-								map_index);
+												mtstate->resultRelInfos[0],
+												map_index);
 			}
 			else
 			{
-				map_index = resultRelInfo - mtstate->resultRelInfo;
+				map_index = mtstate->mt_whichplan;
 				Assert(map_index >= 0 && map_index < mtstate->mt_nplans);
 				tupconv_map = tupconv_map_for_subplan(mtstate, map_index);
 			}
@@ -1632,12 +1632,12 @@ static void
 fireBSTriggers(ModifyTableState *node)
 {
 	ModifyTable *plan = (ModifyTable *) node->ps.plan;
-	ResultRelInfo *resultRelInfo = node->resultRelInfo;
+	ResultRelInfo *resultRelInfo = node->resultRelInfos[0];
 
 	/*
 	 * If the node modifies a partitioned table, we must fire its triggers.
-	 * Note that in that case, node->resultRelInfo points to the first leaf
-	 * partition, not the root table.
+	 * Note that in that case, node->resultRelInfos[0] points to the first
+	 * leaf partition, not the root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		resultRelInfo = node->rootResultRelInfo;
@@ -1683,13 +1683,14 @@ static ResultRelInfo *
 getTargetResultRelInfo(ModifyTableState *node)
 {
 	/*
-	 * Note that if the node modifies a partitioned table, node->resultRelInfo
-	 * points to the first leaf partition, not the root table.
+	 * Note that if the node modifies a partitioned table,
+	 * node->resultRelInfos[0] points to the first leaf partition, not the
+	 * root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		return node->rootResultRelInfo;
 	else
-		return node->resultRelInfo;
+		return node->resultRelInfos[0];
 }
 
 /*
@@ -1910,7 +1911,7 @@ static void
 ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 {
 	ResultRelInfo *targetRelInfo = getTargetResultRelInfo(mtstate);
-	ResultRelInfo *resultRelInfos = mtstate->resultRelInfo;
+	ResultRelInfo **resultRelInfos = mtstate->resultRelInfos;
 	TupleDesc	outdesc;
 	int			numResultRelInfos = mtstate->mt_nplans;
 	int			i;
@@ -1941,7 +1942,7 @@ ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 	for (i = 0; i < numResultRelInfos; ++i)
 	{
 		mtstate->mt_per_subplan_tupconv_maps[i] =
-			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i].ri_RelationDesc),
+			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i]->ri_RelationDesc),
 								   outdesc,
 								   gettext_noop("could not convert row type"));
 	}
@@ -2080,7 +2081,7 @@ ExecModifyTable(PlanState *pstate)
 	}
 
 	/* Preload local variables */
-	resultRelInfo = node->resultRelInfo + node->mt_whichplan;
+	resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 	subplanstate = node->mt_plans[node->mt_whichplan];
 	junkfilter = resultRelInfo->ri_junkFilter;
 
@@ -2118,7 +2119,7 @@ ExecModifyTable(PlanState *pstate)
 
 			if (node->mt_whichplan < node->mt_nplans)
 			{
-				resultRelInfo++;
+				resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 				subplanstate = node->mt_plans[node->mt_whichplan];
 				junkfilter = resultRelInfo->ri_junkFilter;
 				estate->es_result_relation_info = resultRelInfo;
@@ -2309,9 +2310,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	ResultRelInfo *resultRelInfo;
 	Plan	   *subplan;
 	ListCell   *l;
-	int			i;
+	int			i,
+				j;
 	Relation	rel;
 	bool		update_tuple_routing_needed = node->partColsUpdated;
+	Bitmapset  *validsubplans;
 
 	/* check for unsupported flags */
 	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -2328,8 +2331,70 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	mtstate->canSetTag = node->canSetTag;
 	mtstate->mt_done = false;
 
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *partprune;
+
+		ExecAssignExprContext(estate, &mtstate->ps);
+
+		partprune = ExecSetupPartitionPruning(&mtstate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away ModifyTable plans.
+		 */
+		if (!bms_is_empty(partprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
+															list_length(node->plans));
+
+			/*
+			 * If no plans match the given parameters then we must handle this
+			 * case in a special way.  The problem here is that code in
+			 * explain.c requires a ModifyTable to have at least one plan in
+			 * order for it to properly determine the Vars in that plan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set the mt_done flag so that we never
+			 * actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mtstate->mt_done = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->plans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		mtstate->partition_pruning = partprune;
+	}
+	else
+	{
+		nplans = list_length(node->plans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * plans as valid, they must also all be initialized.
+		 */
+		validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		mtstate->partition_pruning = NULL;
+	}
+
+
 	mtstate->mt_plans = (PlanState **) palloc0(sizeof(PlanState *) * nplans);
-	mtstate->resultRelInfo = estate->es_result_relations + node->resultRelIndex;
+	mtstate->resultRelInfos = (ResultRelInfo **)
+		palloc(sizeof(ResultRelInfo *) * nplans);
 
 	/* If modifying a partitioned table, initialize the root table info */
 	if (node->rootResultRelIndex >= 0)
@@ -2353,8 +2418,6 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	 */
 	saved_resultRelInfo = estate->es_result_relation_info;
 
-	resultRelInfo = mtstate->resultRelInfo;
-
 	/*
 	 * mergeTargetRelation must be set if we're running MERGE and mustn't be
 	 * set if we're not.
@@ -2362,13 +2425,20 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	Assert(operation != CMD_MERGE || node->mergeTargetRelation > 0);
 	Assert(operation == CMD_MERGE || node->mergeTargetRelation == 0);
 
-	resultRelInfo->ri_mergeTargetRTI = node->mergeTargetRelation;
-
-	i = 0;
+	j = i = 0;
 	foreach(l, node->plans)
 	{
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
 		subplan = (Plan *) lfirst(l);
 
+		resultRelInfo = estate->es_result_relations + node->resultRelIndex + i;
+		mtstate->resultRelInfos[j] = resultRelInfo;
+
 		/* Initialize the usesFdwDirectModify flag */
 		resultRelInfo->ri_usesFdwDirectModify = bms_is_member(i,
 															  node->fdwDirectModifyPlans);
@@ -2405,7 +2475,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		/* Now init the plan for this result rel */
 		estate->es_result_relation_info = resultRelInfo;
-		mtstate->mt_plans[i] = ExecInitNode(subplan, estate, eflags);
+		mtstate->mt_plans[j] = ExecInitNode(subplan, estate, eflags);
 
 		/* Also let FDWs init themselves for foreign-table result rels */
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
@@ -2421,10 +2491,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 															 eflags);
 		}
 
-		resultRelInfo++;
 		i++;
+		j++;
 	}
 
+	mtstate->resultRelInfos[0]->ri_mergeTargetRTI = node->mergeTargetRelation;
+
 	estate->es_result_relation_info = saved_resultRelInfo;
 
 	/* Get the target relation */
@@ -2477,26 +2549,34 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	/*
 	 * Initialize any WITH CHECK OPTION constraints if needed.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
-	i = 0;
+	j = i = 0;
 	foreach(l, node->withCheckOptionLists)
 	{
-		List	   *wcoList = (List *) lfirst(l);
+		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
 
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
+		wcoList = (List *) lfirst(l);
+
 		foreach(ll, wcoList)
 		{
 			WithCheckOption *wco = (WithCheckOption *) lfirst(ll);
 			ExprState  *wcoExpr = ExecInitQual((List *) wco->qual,
-											   mtstate->mt_plans[i]);
+											   mtstate->mt_plans[j]);
 
 			wcoExprs = lappend(wcoExprs, wcoExpr);
 		}
-
+		resultRelInfo = mtstate->resultRelInfos[j];
 		resultRelInfo->ri_WithCheckOptions = wcoList;
 		resultRelInfo->ri_WithCheckOptionExprs = wcoExprs;
-		resultRelInfo++;
+
+		j++;
 		i++;
 	}
 
@@ -2526,15 +2606,25 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		/*
 		 * Build a projection for each result rel.
 		 */
-		resultRelInfo = mtstate->resultRelInfo;
+		j = i = 0;
 		foreach(l, node->returningLists)
 		{
-			List	   *rlist = (List *) lfirst(l);
+			List	   *rlist;
+
+			if (!bms_is_member(i, validsubplans))
+			{
+				i++;
+				continue;
+			}
+
+			rlist = (List *) lfirst(l);
 
+			resultRelInfo = mtstate->resultRelInfos[j];
 			resultRelInfo->ri_projectReturning =
 				ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps,
 										resultRelInfo->ri_RelationDesc->rd_att);
-			resultRelInfo++;
+			j++;
+			i++;
 		}
 	}
 	else
@@ -2550,7 +2640,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	}
 
 	/* Set the list of arbiter indexes if needed for ON CONFLICT */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (node->onConflictAction != ONCONFLICT_NONE)
 		resultRelInfo->ri_onConflictArbiterIndexes = node->arbiterIndexes;
 
@@ -2654,7 +2744,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		}
 	}
 
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (mtstate->operation == CMD_MERGE)
 		ExecInitMerge(mtstate, estate, resultRelInfo);
 
@@ -2709,11 +2799,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		if (junk_filter_needed)
 		{
-			resultRelInfo = mtstate->resultRelInfo;
 			for (i = 0; i < nplans; i++)
 			{
 				JunkFilter *j;
 
+				resultRelInfo = mtstate->resultRelInfos[i];
 				subplan = mtstate->mt_plans[i]->plan;
 
 				if (operation == CMD_INSERT || operation == CMD_UPDATE)
@@ -2766,13 +2856,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 				}
 
 				resultRelInfo->ri_junkFilter = j;
-				resultRelInfo++;
 			}
 		}
 		else
 		{
 			if (operation == CMD_INSERT)
-				ExecCheckPlanOutput(mtstate->resultRelInfo->ri_RelationDesc,
+				ExecCheckPlanOutput(mtstate->resultRelInfos[0]->ri_RelationDesc,
 									subplan->targetlist);
 		}
 	}
@@ -2819,7 +2908,7 @@ ExecEndModifyTable(ModifyTableState *node)
 	 */
 	for (i = 0; i < node->mt_nplans; i++)
 	{
-		ResultRelInfo *resultRelInfo = node->resultRelInfo + i;
+		ResultRelInfo *resultRelInfo = node->resultRelInfos[i];
 
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
 			resultRelInfo->ri_FdwRoutine != NULL &&
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 739a023965..87339d5e79 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -225,6 +225,7 @@ _copyModifyTable(const ModifyTable *from)
 	COPY_NODE_FIELD(exclRelTlist);
 	COPY_NODE_FIELD(mergeSourceTargetList);
 	COPY_NODE_FIELD(mergeActionList);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e31b6a9c33..b748a1d204 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -393,6 +393,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 	WRITE_NODE_FIELD(mergeSourceTargetList);
 	WRITE_NODE_FIELD(mergeActionList);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 5bf3d28c51..85d7f38d72 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1640,6 +1640,7 @@ _readModifyTable(void)
 	READ_NODE_FIELD(exclRelTlist);
 	READ_NODE_FIELD(mergeSourceTargetList);
 	READ_NODE_FIELD(mergeActionList);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 4e54fe6d25..cd75c59496 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -295,7 +295,8 @@ static ModifyTable *make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam);
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos);
 static GatherMerge *create_gather_merge_plan(PlannerInfo *root,
 						 GatherMergePath *best_path);
 
@@ -2484,6 +2485,7 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 	List	   *subplans = NIL;
 	ListCell   *subpaths,
 			   *subroots;
+	List	   *partpruneinfos = NIL;
 
 	/* Build the plan for each input path */
 	forboth(subpaths, best_path->subpaths,
@@ -2512,6 +2514,27 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (best_path->partitioned_rels != NIL)
+	{
+		int			partrelid = linitial_int(best_path->partitioned_rels);
+		RelOptInfo *rel = root->simple_rel_array[partrelid];
+		List	   *prunequal = NIL;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->resultRelations,
+													  best_path->subpaths, prunequal);
+	}
+
 	plan = make_modifytable(root,
 							best_path->operation,
 							best_path->canSetTag,
@@ -2527,7 +2550,8 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 							best_path->onconflict,
 							best_path->mergeSourceTargetList,
 							best_path->mergeActionList,
-							best_path->epqParam);
+							best_path->epqParam,
+							partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, &best_path->path);
 
@@ -6600,7 +6624,8 @@ make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam)
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos)
 {
 	ModifyTable *node = makeNode(ModifyTable);
 	List	   *fdw_private_list;
@@ -6662,6 +6687,7 @@ make_modifytable(PlannerInfo *root,
 	node->mergeSourceTargetList = mergeSourceTargetList;
 	node->mergeActionList = mergeActionList;
 	node->epqParam = epqParam;
+	node->part_prune_infos = partpruneinfos;
 
 	/*
 	 * For each result relation that is a foreign table, allow the FDW to
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 421dc79cc4..5cfc665347 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1197,6 +1197,7 @@ inheritance_planner(PlannerInfo *root)
 	Query	   *parent_parse;
 	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
 	PlannerInfo **parent_roots = NULL;
+	PlannerInfo *partition_root = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1274,6 +1275,32 @@ inheritance_planner(PlannerInfo *root)
 		 * the ModifyTable node, if one is needed at all.
 		 */
 		partitioned_relids = bms_make_singleton(top_parentRTindex);
+
+		/*
+		 * For partitioned tables, since we're able to determine the minimum
+		 * set of partitions required much more easily than what we can do
+		 * with an inheritance hierarchy, we invoke the grouping_planner on
+		 * the entire given query in order to determine the minimum set of
+		 * partitions which will be required below.  This may mean that we
+		 * invoke the grouping planner far fewer times, as otherwise we'd have
+		 * to invoke it once for each partition.
+		 */
+
+		/*
+		 * Since the planner tends to scribble on the parse, we must make a
+		 * copy of it.  We also must make copies of the PlannerInfo and
+		 * PlannerGlobal since these will also be modified from the call to
+		 * grouping_planner.
+		 */
+		partition_root = makeNode(PlannerInfo);
+		partition_root->glob = makeNode(PlannerGlobal);
+
+		memcpy(partition_root, root, sizeof(PlannerInfo));
+		memcpy(partition_root->glob, root->glob, sizeof(PlannerGlobal));
+
+		partition_root->parse = copyObject(partition_root->parse);
+
+		grouping_planner(partition_root, true, 0.0 /* retrieve all tuples */ );
 	}
 
 	/*
@@ -1304,6 +1331,21 @@ inheritance_planner(PlannerInfo *root)
 		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
+		/*
+		 * If the target rel is a partitioned table then skip any child
+		 * partitions which were found to be dummies by the grouping_planner
+		 * call performed above.
+		 */
+		if (partition_root)
+		{
+			RelOptInfo *rel;
+
+			rel = find_base_rel(partition_root, appinfo->child_relid);
+
+			if (IS_DUMMY_REL(rel))
+				continue;
+		}
+
 		/*
 		 * expand_inherited_rtentry() always processes a parent before any of
 		 * that parent's children, so the parent_root for this relation should
@@ -1629,6 +1671,23 @@ inheritance_planner(PlannerInfo *root)
 		Assert(list_length(partitioned_rels) >= 1);
 	}
 
+	/*
+	 * The individual grouping_planner calls per partition above performed no
+	 * planning on the actual partitioned tables, however, in order to allow
+	 * partition pruning at run-time we must know the baserestrictinfo of each
+	 * partition.  We simply replace the RelOptInfos from the initial full
+	 * plan which was generated and replace the non-complete RelOptInfos which
+	 * are stored in root.
+	 */
+	if (partition_root)
+	{
+		int			i;
+
+		i = -1;
+		while ((i = bms_next_member(partitioned_relids, i)) >= 0)
+			root->simple_rel_array[i] = partition_root->simple_rel_array[i];
+	}
+
 	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
 	add_path(final_rel, (Path *)
 			 create_modifytable_path(root, final_rel,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1986abaa9c..526bc2019f 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1027,6 +1027,8 @@ typedef struct EPQState
 } EPQState;
 
 
+struct PartitionPruning;
+
 /* ----------------
  *	 ResultState information
  * ----------------
@@ -1083,7 +1085,7 @@ typedef struct ModifyTableState
 	PlanState **mt_plans;		/* subplans (one per target rel) */
 	int			mt_nplans;		/* number of plans in the array */
 	int			mt_whichplan;	/* which one is being executed (0..n-1) */
-	ResultRelInfo *resultRelInfo;	/* per-subplan target relations */
+	ResultRelInfo **resultRelInfos; /* per-subplan target relations */
 	ResultRelInfo *rootResultRelInfo;	/* root target relation (partitioned
 										 * table root) */
 	List	  **mt_arowmarks;	/* per-subplan ExecAuxRowMark lists */
@@ -1109,6 +1111,12 @@ typedef struct ModifyTableState
 
 	/* Flags showing which subcommands are present INS/UPD/DEL/DO NOTHING */
 	int			mt_merge_subcommands;
+
+	/*
+	 * Details required to allow partitions to be eliminated from the scan, or
+	 * NULL if not possible.
+	 */
+	struct PartitionPruning *partition_pruning;
 } ModifyTableState;
 
 /* ----------------
@@ -1130,7 +1138,6 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
-struct PartitionPruning;
 
 struct AppendState
 {
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index a7dbd31466..c1b6c21f70 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -242,6 +242,8 @@ typedef struct ModifyTable
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 	List	   *mergeSourceTargetList;
 	List	   *mergeActionList;	/* actions for MERGE */
+	List	   *part_prune_infos;	/* Mapping details for run-time subplan
+									 * pruning, one per partitioned_rels */
 } ModifyTable;
 
 /* ----------------
-- 
2.16.2.windows.1

#111Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#108)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/05 12:14, Amit Langote wrote:

I will post comments on your v19 later today.

I looked at it and here are my thoughts on it after having for the first
time looked very closely at it.

* Regarding PartitionPruneInfo:

I think the names of the fields could be improved -- like pruning_steps
instead prunesteps, unpruned_parts instead of allpartindexs. The latter
is even a bit misleading because it doesn't in fact contain *all*
partition indexes, only those that are unpruned, because each either has a
subpath or it appears in (unpruned) partitioned_rels list. Also, I didn't
like the name subnodeindex and subpartindex very much. subplan_indexes
and parent_indexes would sound more informative to me.

* make_partition_pruneinfo has a parameter resultRelations that's not used
anywhere

* In make_partition_pruneinfo()

allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);

I think these arrays need to have root->simple_rel_array_size + 1
elements, because they're subscripted using RT indexes which start at 1.

* Also in make_partition_pruneinfo()

/* Initialize to -1 to indicate the rel was not found */
for (i = 0; i < root->simple_rel_array_size; i++)
{
allsubnodeindex[i] = -1;
allsubpartindex[i] = -1;
}

Maybe, allocate the arrays above mentioned using palloc0 and don't do this
initialization. Instead make the indexes that are stored in these start
with 1 and consider 0 as invalid entries.

* Regarding the code added in execPartition.c and execPartition.h:

I wondered why PartitionedRelPruning is named the way it is. I saw many
parallels with PartitionDispatchData both in terms of the main thing it
consists of, that is, the map that translates partition indexes as in
partition descriptor to that of subplans or of some other executor
structure. Also, I imagine you tried to mimic PartitionTupleRouting with
PartitionPruning but not throughout. For example, tuple routing struct
pointer variables are throughout called proute, whereas PartitionPruning
ones are called partprune instead of, say, pprune. Consistency would
help, imho.

* Instead of nesting PartitionedRelPruning inside another, just store them
in a global flat array in the PartitionPruning, like PartitionTupleRouting
does for PartitionDispatch of individual partitioned tables in the tree.

typedef struct PartitionedRelPruning
{
int nparts;
int *subnodeindex;
struct PartitionedRelPruning **subpartprune;

* I don't see why there needs to be nparts in the above, because it
already has a PartitionPruneContext member which has that information.

In fact, I made most of changes myself while going through the code.
Please see attached the delta patch. It also tweaks quite a few other
things including various comments. I think parts of it apply to 0001,
0003, and 0004 patches. See if this looks good to you.

Thanks,
Amit

Attachments:

v19-delta-amit.patchtext/plain; charset=UTF-8; name=v19-delta-amit.patchDownload
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 17da8cdbd3..1041871e51 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,11 +39,11 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
-static void find_subplans_for_extparams_recurse(
-									PartitionedRelPruning *partrelprune,
+static void find_subplans_for_extparams_recurse(PartitionPruningDispatch *all_ppd,
+									int dispatch_offset,
 									Bitmapset **validsubplans);
-static void find_subplans_for_allparams_recurse(
-									PartitionedRelPruning *partrelprune,
+static void find_subplans_for_allparams_recurse(PartitionPruningDispatch *all_ppd,
+									int dispatch_offset,
 									Bitmapset **validsubplans);
 
 
@@ -1343,27 +1343,27 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 PartitionPruning *
 ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
 {
-	PartitionedRelPruning *partrelprunes;
-	PartitionPruning *partprune;
+	PartitionPruning *pprune;
+	PartitionPruningDispatch *all_ppd;
 	ListCell   *lc;
 	int			i;
 
 	Assert(partitionpruneinfo != NIL);
 
-	partprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
-	partrelprunes = (PartitionedRelPruning *)
-					palloc(sizeof(PartitionedRelPruning) *
-						   list_length(partitionpruneinfo));
+	pprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	all_ppd = pprune->partition_dispatch_info =
+				(PartitionPruningDispatchData **)
+						palloc(sizeof(PartitionPruningDispatchData *) *
+							   list_length(partitionpruneinfo));
 
 	/*
 	 * The first item in the array contains the details for the query's target
 	 * partition, so record that as the root of the partition hierarchy.
 	 */
-	partprune->partrelpruning = partrelprunes;
-	partprune->npartrelpruning = list_length(partitionpruneinfo);
-	partprune->extparams = NULL;
-	partprune->execparams = NULL;
-	partprune->allparams = NULL;
+	pprune->num_dispatch = list_length(partitionpruneinfo);
+	pprune->extparams = NULL;
+	pprune->execparams = NULL;
+	pprune->allparams = NULL;
 
 	/*
 	 * Create a sub memory context which we'll use when making calls to the
@@ -1372,79 +1372,70 @@ ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
 	 * call the function in this context to avoid any memory leaking in the
 	 * executor's memory context.
 	 */
-	partprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
-													 "Partition Prune",
-													 ALLOCSET_DEFAULT_SIZES);
+	pprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+												  "Partition Prune",
+												  ALLOCSET_DEFAULT_SIZES);
 
 	i = 0;
 	foreach(lc, partitionpruneinfo)
 	{
 		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
-		PartitionedRelPruning *partrelprune = &partrelprunes[i];
-		PartitionPruneContext *context = &partrelprune->context;
+		PartitionPruningDispatch *ppd = &all_ppd[i];
+		PartitionPruneContext *context;
 		PartitionDesc		partdesc;
 		Relation			rel;
 		PartitionKey		partkey;
 		int					partnatts;
-		int		j;
 
-		partrelprune->allpartindexes = bms_copy(pinfo->allpartindexes);
-		partrelprune->nparts = pinfo->nparts;
-		partrelprune->subnodeindex = palloc(sizeof(int) * pinfo->nparts);
-		partrelprune->subpartprune = palloc(sizeof(PartitionedRelPruning *) *
-										 pinfo->nparts);
+		*ppd = (PartitionPruningDispatch)
+								palloc(sizeof(PartitionPruningDispatchData));
+		context = &((*ppd)->context);
+		(*ppd)->pruning_steps = pinfo->pruning_steps;
+		(*ppd)->extparams = bms_copy(pinfo->extparams);
+		(*ppd)->allparams = bms_union(pinfo->extparams, pinfo->execparams);
+		(*ppd)->unpruned_parts = bms_copy(pinfo->unpruned_parts);
+
+		/* Initialize PartitionPruneContext struct. */
 
 		/*
-		 * We must make a copy of this rather than pointing directly to the
-		 * plan's version as we may end up making modifications to it later.
+		 * Note that the relation must've been already locked in
+		 * ExecLockNonLeafAppendTables()
 		 */
-		memcpy(partrelprune->subnodeindex, pinfo->subnodeindex,
-			   sizeof(int) * pinfo->nparts);
-
-		for (j = 0; j < pinfo->nparts; j++)
-		{
-			int subpartidx = pinfo->subpartindex[j];
-
-			Assert(subpartidx < list_length(partitionpruneinfo));
-
-			if (subpartidx >= 0)
-				partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
-			else
-				partrelprune->subpartprune[j] = NULL;
-		}
-
 		rel = relation_open(pinfo->reloid, NoLock);
-
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
 
 		context->strategy = partkey->strategy;
 		context->partnatts = partnatts = partkey->partnatts;
-
 		context->partopcintype = partkey->partopcintype;
 		context->partopfamily = partkey->partopfamily;
 		context->partcollation = partkey->partcollation;
 		context->partsupfunc = partkey->partsupfunc;
-		context->nparts = pinfo->nparts;
+		context->nparts = partdesc->nparts;
 		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
 
 		context->planstate = planstate;
 		context->safeparams = NULL;		/* empty for now */
 
-		partrelprune->prunesteps = pinfo->prunesteps;
+		(*ppd)->subplan_indexes = palloc(sizeof(int) * context->nparts);
+		(*ppd)->parent_indexes = palloc(sizeof(int) * context->nparts);
 
-		partrelprune->extparams = bms_copy(pinfo->extparams);
-		partrelprune->allparams = bms_union(pinfo->extparams,
-											pinfo->execparams);
-
-		partprune->extparams = bms_add_members(partprune->extparams,
-											   pinfo->extparams);
-
-		partprune->execparams = bms_add_members(partprune->execparams,
-												pinfo->execparams);
+		/*
+		 * Copy from the planner's arrays as is, for now.  This might change
+		 * after performing pruning with external params that's done at the
+		 * plan startup.
+		 */
+		memcpy((*ppd)->subplan_indexes, pinfo->subplan_indexes,
+			   sizeof(int) * context->nparts);
+		memcpy((*ppd)->parent_indexes, pinfo->parent_indexes,
+			   sizeof(int) * context->nparts);
 
+		/* Add the params of this pinfo to the global bitmapset in pprune. */
+		pprune->extparams = bms_add_members(pprune->extparams,
+											pinfo->extparams);
+		pprune->execparams = bms_add_members(pprune->execparams,
+											 pinfo->execparams);
 		relation_close(rel, NoLock);
-
 		i++;
 	}
 
@@ -1452,16 +1443,15 @@ ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
 	 * Cache the union of the Param ids of both types.  This saves having to
 	 * recalculate it everytime we need to know what they are.
 	 */
-	partprune->allparams = bms_union(partprune->extparams,
-									 partprune->execparams);
+	pprune->allparams = bms_union(pprune->extparams, pprune->execparams);
 
-	return partprune;
+	return pprune;
 }
 
 /*
  * ExecFindInitialMatchingSubPlans
  *		Determine which subset of subplan nodes we need to initialize based
- *		on the details stored in 'partprune'.  Here we only determine the
+ *		on the details stored in 'pprune'.  Here we only determine the
  *		matching partitions using values known during plan startup, which is
  *		only external Params.  Exec Params will be unknown at this time.  We
  *		must delay pruning using exec Params until the actual executor run.
@@ -1473,14 +1463,13 @@ ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
  * return its matching subnode indexes assuming that the caller discarded
  * the original non-matching subnodes.
  *
- * This function must only be called if 'partprune' has any extparams.
+ * This function must only be called if 'pprune' has any extparams.
  *
  * 'nsubplans' must be passed as the total number of unpruned subplans.
  */
 Bitmapset *
-ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
+ExecFindInitialMatchingSubPlans(PartitionPruning *pprune, int nsubplans)
 {
-	PartitionedRelPruning *partrelprune;
 	MemoryContext		oldcontext;
 	Bitmapset		   *result = NULL;
 
@@ -1488,25 +1477,27 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
 	 * Ensure there's actually external params, or we've not been called
 	 * already.
 	 */
-	Assert(!bms_is_empty(partprune->extparams));
-
-	partrelprune = partprune->partrelpruning;
+	Assert(!bms_is_empty(pprune->extparams));
 
 	/*
 	 * Switch to a temp context to avoid leaking memory in the
 	 * executor's memory context.
 	 */
-	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
 
-	/* Determine which subplans match these external params */
-	find_subplans_for_extparams_recurse(partrelprune, &result);
+	/*
+	 * Determine which subplans match using external params.  We ask it to
+	 * start pruning with 0th partitioned table, that is, the root.
+	 */
+	find_subplans_for_extparams_recurse(pprune->partition_dispatch_info, 0,
+										&result);
 
 	MemoryContextSwitchTo(oldcontext);
 
 	/* Move to the correct memory context */
 	result = bms_copy(result);
 
-	MemoryContextReset(partprune->prune_context);
+	MemoryContextReset(pprune->prune_context);
 
 	/*
 	 * Record that partition pruning has been performed for external params.
@@ -1514,8 +1505,8 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
 	 * with the same input and also so that ExecFindMatchingSubPlans is aware
 	 * that pruning has already been done for external Params.
 	 */
-	bms_free(partprune->extparams);
-	partprune->extparams = NULL;
+	bms_free(pprune->extparams);
+	pprune->extparams = NULL;
 
 	/*
 	 * If any subplans were pruned, we must re-sequence the subplan indexes so
@@ -1532,7 +1523,7 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
 		 * First we must build a map which allows us to map the old subplan
 		 * index into the new one.
 		 */
-		subplanidxmap = (int *) palloc(sizeof(int) * nsubplans);
+		subplanidxmap = (int *) palloc0(sizeof(int) * nsubplans);
 		newidx = 0;
 		for (i = 0; i < nsubplans; i++)
 		{
@@ -1543,28 +1534,28 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
 		}
 
 		/*
-		 * Now we can re-sequence each PartitionPruneInfo's subnodeindex
-		 * so that they point to the new index of the subnode.
+		 * Now we can re-sequence each PartitionPruningDispatch's indexes
+		 * so that they point to the new indexes of subplans.
 		 */
-		for (i = 0; i < partprune->npartrelpruning; i++)
+		for (i = 0; i < pprune->num_dispatch; i++)
 		{
-			PartitionedRelPruning *partrelprune;
+			PartitionPruningDispatch ppd;
 			int j;
 
-			partrelprune = &partprune->partrelpruning[i];
+			ppd = pprune->partition_dispatch_info[i];
 
 			/*
-			 * We also need to reset the allpartindexes field so that it
+			 * We also need to reset the unpruned_parts field so that it
 			 * only contains partition indexes that we actually still have
-			 * subnodeindexes for.  It seems easier to build a fresh one,
-			 * rather than trying to update the existing one.
+			 * a valid value in subplan_indexes for.  It seems easier to build
+			 * a fresh one, rather than trying to update the existing one.
 			 */
-			bms_free(partrelprune->allpartindexes);
-			partrelprune->allpartindexes = NULL;
+			bms_free(ppd->unpruned_parts);
+			ppd->unpruned_parts = NULL;
 
-			for (j = 0; j < partrelprune->nparts; j++)
+			for (j = 0; j < ppd->context.nparts; j++)
 			{
-				int oldidx = partrelprune->subnodeindex[j];
+				int oldidx = ppd->subplan_indexes[j];
 
 				/*
 				 * If this partition existed as a subplan then change the old
@@ -1575,12 +1566,11 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
 				 */
 				if (oldidx >= 0)
 				{
-					partrelprune->subnodeindex[j] = subplanidxmap[oldidx];
+					ppd->subplan_indexes[j] = subplanidxmap[oldidx];
 
-					if (subplanidxmap[oldidx] >= 0)
-						partrelprune->allpartindexes =
-								bms_add_member(partrelprune->allpartindexes,
-											   j);
+					if (ppd->subplan_indexes[j] >= 0)
+						ppd->unpruned_parts =
+								bms_add_member(ppd->unpruned_parts, j);
 				}
 			}
 		}
@@ -1597,10 +1587,12 @@ ExecFindInitialMatchingSubPlans(PartitionPruning *partprune, int nsubplans)
  *		Recursive worker function for ExecFindInitialMatchingSubPlans.
  */
 static void
-find_subplans_for_extparams_recurse(PartitionedRelPruning *partrelprune,
+find_subplans_for_extparams_recurse(PartitionPruningDispatch *all_ppd,
+									int dispatch_offset,
 									Bitmapset **validsubplans)
 {
-	PartitionPruneContext *context = &partrelprune->context;
+	PartitionPruningDispatch ppd = all_ppd[dispatch_offset];
+	PartitionPruneContext *context = &ppd->context;
 	Bitmapset	   *partset;
 	int			i;
 
@@ -1616,23 +1608,29 @@ find_subplans_for_extparams_recurse(PartitionedRelPruning *partrelprune,
 	 * any subpartitioned tables as we may find their partition keys match
 	 * some Params at their level.
 	 */
-	if (!bms_is_empty(partrelprune->extparams))
+	if (!bms_is_empty(ppd->extparams))
 	{
-		context->safeparams = partrelprune->extparams;
-		partset = get_matching_partitions(context, partrelprune->prunesteps);
+		context->safeparams = ppd->extparams;
+		partset = get_matching_partitions(context, ppd->pruning_steps);
 	}
 	else
-		partset = partrelprune->allpartindexes;
+		partset = ppd->unpruned_parts;
 
 	/* Translate partset into subnode indexes */
 	i = -1;
 	while ((i = bms_next_member(partset, i)) >= 0)
 	{
-		if (partrelprune->subnodeindex[i] >= 0)
+		/* If the chosen partition is a leaf partition, add its subplan. */
+		if (ppd->subplan_indexes[i] >= 0)
 			*validsubplans = bms_add_member(*validsubplans,
-											partrelprune->subnodeindex[i]);
-		else if (partrelprune->subpartprune[i] != NULL)
-			find_subplans_for_extparams_recurse(partrelprune->subpartprune[i],
+											ppd->subplan_indexes[i]);
+		/*
+		 * Else if it's a partitioned table, recurse to perform pruning
+		 * for its own partitions.
+		 */
+		else if (ppd->parent_indexes[i])
+			find_subplans_for_extparams_recurse(all_ppd,
+												ppd->parent_indexes[i],
 												validsubplans);
 		else
 		{
@@ -1650,31 +1648,34 @@ find_subplans_for_extparams_recurse(PartitionedRelPruning *partrelprune,
 /*
  * ExecFindMatchingSubPlans
  *		Determine which subplans match the the pruning steps detailed in
- *		'partprune' for the current Param values.
+ *		'pprune' for the current Param values.
  */
 Bitmapset *
-ExecFindMatchingSubPlans(PartitionPruning *partprune)
+ExecFindMatchingSubPlans(PartitionPruning *pprune)
 {
-	PartitionedRelPruning *partrelprune;
 	MemoryContext		oldcontext;
 	Bitmapset *result = NULL;
 
-	partrelprune = partprune->partrelpruning;
-
 	/*
 	 * Switch to a temp context to avoid leaking memory in the
 	 * executor's memory context.
 	 */
-	oldcontext = MemoryContextSwitchTo(partprune->prune_context);
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
 
-	find_subplans_for_allparams_recurse(partrelprune, &result);
+	/*
+	 * Determine which subplans match using all the params, including both
+	 * the external and executor params.  We ask it to start pruning with
+	 * 0th partitioned table, that is, the root.
+	 */
+	find_subplans_for_allparams_recurse(pprune->partition_dispatch_info, 0,
+										&result);
 
 	MemoryContextSwitchTo(oldcontext);
 
 	/* Move to the correct memory context */
 	result = bms_copy(result);
 
-	MemoryContextReset(partprune->prune_context);
+	MemoryContextReset(pprune->prune_context);
 
 	return result;
 }
@@ -1684,10 +1685,12 @@ ExecFindMatchingSubPlans(PartitionPruning *partprune)
  *		Recursive worker function for ExecFindMatchingSubPlans.
  */
 static void
-find_subplans_for_allparams_recurse(PartitionedRelPruning *partrelprune,
+find_subplans_for_allparams_recurse(PartitionPruningDispatch *all_ppd,
+									int dispatch_offset,
 									Bitmapset **validsubplans)
 {
-	PartitionPruneContext *context = &partrelprune->context;
+	PartitionPruningDispatch ppd = all_ppd[dispatch_offset];
+	PartitionPruneContext *context = &ppd->context;
 	Bitmapset	   *partset;
 	int			i;
 
@@ -1703,23 +1706,30 @@ find_subplans_for_allparams_recurse(PartitionedRelPruning *partrelprune,
 	 * any subpartitioned tables as we may find their partition keys match
 	 * some Params at their level.
 	 */
-	if (!bms_is_empty(partrelprune->allparams))
+	if (!bms_is_empty(ppd->allparams))
 	{
-		context->safeparams = partrelprune->allparams;
-		partset = get_matching_partitions(context, partrelprune->prunesteps);
+		context->safeparams = ppd->allparams;
+		partset = get_matching_partitions(context, ppd->pruning_steps);
 	}
 	else
-		partset = partrelprune->allpartindexes;
+		partset = ppd->unpruned_parts;
 
 	/* Translate partset into subnode indexes */
 	i = -1;
 	while ((i = bms_next_member(partset, i)) >= 0)
 	{
-		if (partrelprune->subnodeindex[i] >= 0)
-			*validsubplans = bms_add_member(*validsubplans,
-											partrelprune->subnodeindex[i]);
-		else if (partrelprune->subpartprune[i] != NULL)
-			find_subplans_for_allparams_recurse(partrelprune->subpartprune[i],
+		/* If the chosen partition is a leaf partition, add its subplan. */
+		if (ppd->subplan_indexes[i] >= 0)
+			*validsubplans =
+						bms_add_member(*validsubplans,
+									   ppd->subplan_indexes[i]);
+		/*
+		 * Else if it's a partitioned table, recurse to perform pruning
+		 * for its own partitions.
+		 */
+		else if (ppd->parent_indexes[i] >= 0)
+			find_subplans_for_allparams_recurse(all_ppd,
+												ppd->parent_indexes[i],
 												validsubplans);
 		else
 		{
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 5286ada835..c589be4563 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -126,22 +126,22 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	/* If run-time partition pruning is enabled, then setup that up now */
 	if (node->part_prune_infos != NIL)
 	{
-		PartitionPruning *partprune;
+		PartitionPruning *pprune;
 
 		ExecAssignExprContext(estate, &appendstate->ps);
 
-		partprune = ExecSetupPartitionPruning(&appendstate->ps,
-											  node->part_prune_infos);
+		pprune = ExecSetupPartitionPruning(&appendstate->ps,
+										   node->part_prune_infos);
 
 		/*
 		 * When there are external params matching the partition key we may be
 		 * able to prune away Append subplans now.
 		 */
-		if (!bms_is_empty(partprune->extparams))
+		if (!bms_is_empty(pprune->extparams))
 		{
 			/* Determine which subplans match the external params */
-			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
-											list_length(node->appendplans));
+			nplans = list_length(node->appendplans);
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune, nplans);
 
 			/*
 			 * If no subplans match the given parameters then we must handle
@@ -173,10 +173,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 		 * If there's no exec params then no further pruning can be done, we
 		 * can just set the valid subplans to all remaining subplans.
 		 */
-		if (bms_is_empty(partprune->execparams))
+		if (bms_is_empty(pprune->execparams))
 			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
 
-		appendstate->partition_pruning = partprune;
+		appendstate->partition_pruning = pprune;
 
 	}
 	else
diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index d9cf911f4f..cf2c1ec6dc 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -91,22 +91,22 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/* If run-time partition pruning is enabled, then setup that up now */
 	if (node->part_prune_infos != NIL)
 	{
-		PartitionPruning *partprune;
+		PartitionPruning *pprune;
 
 		ExecAssignExprContext(estate, &mergestate->ps);
 
-		partprune = ExecSetupPartitionPruning(&mergestate->ps,
-											  node->part_prune_infos);
+		pprune = ExecSetupPartitionPruning(&mergestate->ps,
+										   node->part_prune_infos);
 
 		/*
 		 * When there are external params matching the partition key we may be
 		 * able to prune away MergeAppend subplans now.
 		 */
-		if (!bms_is_empty(partprune->extparams))
+		if (!bms_is_empty(pprune->extparams))
 		{
 			/* Determine which subplans match the external params */
-			validsubplans = ExecFindInitialMatchingSubPlans(partprune,
-											list_length(node->mergeplans));
+			nplans = list_length(node->mergeplans);
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune, nplans);
 
 			/*
 			 * If no subplans match the given parameters then we must handle
@@ -140,13 +140,13 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 		 * Otherwise we set the valid subplans to NULL so that they can be
 		 * determined during actual execution.
 		 */
-		if (bms_is_empty(partprune->execparams))
+		if (bms_is_empty(pprune->execparams))
 			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
 		else
 			mergestate->ms_valid_subplans = NULL;
 
 
-		mergestate->partition_pruning = partprune;
+		mergestate->partition_pruning = pprune;
 
 	}
 	else
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 739a023965..722e79be5b 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2175,11 +2175,11 @@ _copyPartitionPruneInfo(const PartitionPruneInfo *from)
 	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
 
 	COPY_SCALAR_FIELD(reloid);
-	COPY_NODE_FIELD(prunesteps);
-	COPY_BITMAPSET_FIELD(allpartindexes);
+	COPY_NODE_FIELD(pruning_steps);
+	COPY_BITMAPSET_FIELD(unpruned_parts);
 	COPY_SCALAR_FIELD(nparts);
-	COPY_POINTER_FIELD(subnodeindex, from->nparts * sizeof(int));
-	COPY_POINTER_FIELD(subpartindex, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subplan_indexes, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(parent_indexes, from->nparts * sizeof(int));
 	COPY_BITMAPSET_FIELD(extparams);
 	COPY_BITMAPSET_FIELD(execparams);
 
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e31b6a9c33..88e7f08551 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1757,17 +1757,17 @@ _outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
 	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
 
 	WRITE_OID_FIELD(reloid);
-	WRITE_NODE_FIELD(prunesteps);
-	WRITE_BITMAPSET_FIELD(allpartindexes);
+	WRITE_NODE_FIELD(pruning_steps);
+	WRITE_BITMAPSET_FIELD(unpruned_parts);
 	WRITE_INT_FIELD(nparts);
 
-	appendStringInfoString(str, " :subnodeindex");
+	appendStringInfoString(str, " :subplan_indexes");
 	for (i = 0; i < node->nparts; i++)
-		appendStringInfo(str, " %d", node->subnodeindex[i]);
+		appendStringInfo(str, " %d", node->subplan_indexes[i]);
 
-	appendStringInfoString(str, " :subpartindex");
+	appendStringInfoString(str, " :parent_indexes");
 	for (i = 0; i < node->nparts; i++)
-		appendStringInfo(str, " %d", node->subpartindex[i]);
+		appendStringInfo(str, " %d", node->parent_indexes[i]);
 
 	WRITE_BITMAPSET_FIELD(extparams);
 	WRITE_BITMAPSET_FIELD(execparams);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 5bf3d28c51..6e059ec568 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1363,11 +1363,11 @@ _readPartitionPruneInfo(void)
 	READ_LOCALS(PartitionPruneInfo);
 
 	READ_OID_FIELD(reloid);
-	READ_NODE_FIELD(prunesteps);
-	READ_BITMAPSET_FIELD(allpartindexes);
+	READ_NODE_FIELD(pruning_steps);
+	READ_BITMAPSET_FIELD(unpruned_parts);
 	READ_INT_FIELD(nparts);
-	READ_INT_ARRAY(subnodeindex, local_node->nparts);
-	READ_INT_ARRAY(subpartindex, local_node->nparts);
+	READ_INT_ARRAY(subplan_indexes, local_node->nparts);
+	READ_INT_ARRAY(parent_indexes, local_node->nparts);
 	READ_BITMAPSET_FIELD(extparams);
 	READ_BITMAPSET_FIELD(execparams);
 
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d6c94846d3..c33e0ab206 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1107,9 +1107,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		 * and allow translation of partition indexes into subpath indexes.
 		 */
 		if (prunequal != NIL)
-			partpruneinfos = make_partition_pruneinfo(root,
-											best_path->partitioned_rels, NIL,
-											best_path->subpaths, prunequal);
+			partpruneinfos =
+					make_partition_pruneinfo(root,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
 	}
 
 	/*
@@ -1257,9 +1258,10 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		 * and allow translation of partition indexes into subpath indexes.
 		 */
 		if (prunequal != NIL)
-			partpruneinfos = make_partition_pruneinfo(root,
-											best_path->partitioned_rels, NIL,
-											best_path->subpaths, prunequal);
+			partpruneinfos =
+					make_partition_pruneinfo(root,
+											 best_path->partitioned_rels,
+											 best_path->subpaths, prunequal);
 	}
 
 	node->partitioned_rels = best_path->partitioned_rels;
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index e8815cb697..5483023dc2 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -238,131 +238,113 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 
 /*
  * make_partition_pruneinfo
- *		Return a List of PartitionPruneInfos, one for each 'partitioned_rel',
- *		or NIL if no Params were found matching the partition key, in which
- *		case run-time partition pruning is useless.
+ *		Return a List of PartitionPruneInfos, one for each relation in
+ *		'partitioned_rel' or NIL if no Params matching the partition key
+ *		were found, in which case, run-time partition pruning is useless.
  *
- * Here we index the subpaths by partition index so that we're able to
- * translate the output of get_matching_partitions into subpath indexes to
- * possibly allow for further partition pruning to be performed during
- * execution.
+ * Each PartitionPruneInfo node consists of a map to translate the partition
+ * indexes as output by get_matching_partitions to the indexes of their
+ * corresponding subplans in the array of subplans that contains entries
+ * corresponding to all the partitions in the tree that are selected by the
+ * planner.
  */
 List *
-make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
-						 List *resultRelations, List *subpaths,
-						 List *prunequal)
+make_partition_pruneinfo(PlannerInfo *root, List *partitioned_rels,
+						 List *subpaths, List *prunequal)
 {
-	RangeTblEntry	   *rte;
-	RelOptInfo		   *parentpart;
 	ListCell		   *lc;
 	List			   *pinfolist = NIL;
-	int				   *allsubnodeindex;
-	int				   *allsubpartindex;
+	int				   *all_subpath_indexes;
+	int				   *all_parent_indexes;
 	int					i;
 	bool				gotparam = false;
 
 	/*
-	 * Allocate two arrays, one to allow quick lookups of the 'subpaths' index
-	 * of a relation by relid and another to lookup the 'partitioned_rel'
-	 * index by relid.
+	 * Create a mapping from RT indexes of the subpaths' relations to their
+	 * ordinal position in the list.  Same for the RT indexes appearing in
+	 * partitioned_rels list.  Note that RT indexes start with 1, so must
+	 * allocate space for root->simple_rel_array_size + 1 integers.
 	 */
-	allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
-	allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);
-
-	/* Initialize to -1 to indicate the rel was not found */
-	for (i = 0; i < root->simple_rel_array_size; i++)
-	{
-		allsubnodeindex[i] = -1;
-		allsubpartindex[i] = -1;
-	}
+	all_subpath_indexes = palloc0(sizeof(int) *
+								  root->simple_rel_array_size + 1);
+	all_parent_indexes = palloc0(sizeof(int) *
+								 root->simple_rel_array_size + 1);
 
 	/*
 	 * Now loop over each subpath and fill in the index of the subpath for the
-	 * subpath's relid.
+	 * subpath's relid.  Make the indexes start at 1, so all 0 entries in the
+	 * array would mean a relation that is not in this partition tree or is
+	 * pruned.
 	 */
-	if (resultRelations != NIL)
+	i = 1;
+	foreach(lc, subpaths)
 	{
-		i = 0;
-		foreach(lc, resultRelations)
-		{
-			int resultrel = lfirst_int(lc);
-			Assert(resultrel < root->simple_rel_array_size);
-			allsubnodeindex[resultrel] = i;
-			i++;
-		}
-	}
-	else
-	{
-		i = 0;
-		foreach(lc, subpaths)
-		{
-			Path *path = (Path *) lfirst(lc);
-			RelOptInfo *pathrel = path->parent;
+		Path *path = (Path *) lfirst(lc);
+		RelOptInfo *pathrel = path->parent;
 
-			Assert(IS_SIMPLE_REL(pathrel));
-			Assert(pathrel->relid < root->simple_rel_array_size);
+		Assert(IS_SIMPLE_REL(pathrel));
+		Assert(pathrel->relid < root->simple_rel_array_size);
 
-			allsubnodeindex[pathrel->relid] = i;
-			i++;
-		}
+		all_subpath_indexes[pathrel->relid] = i;
+		i++;
 	}
 
-	/* Likewise for the partition_rels */
-	i = 0;
-	foreach(lc, partition_rels)
+	/* Likewise for the partitioned_rels */
+	i = 1;
+	foreach(lc, partitioned_rels)
 	{
 		Index	rti = lfirst_int(lc);
 
 		Assert(rti < root->simple_rel_array_size);
-
-		allsubpartindex[rti] = i;
+		all_parent_indexes[rti] = i;
 		i++;
 	}
 
-	/* We now build a PartitionPruneInfo for each partition_rels */
+	/* We now build a PartitionPruneInfo for each partitioned_rels */
 	i = 0;
-	foreach(lc, partition_rels)
+	foreach(lc, partitioned_rels)
 	{
+		RangeTblEntry	   *rte;
+		RelOptInfo		   *parent;
 		Index		rti = lfirst_int(lc);
-		RelOptInfo *subpart = find_base_rel(root, rti);
+		RelOptInfo *rel = find_base_rel(root, rti);
 		PartitionPruneInfo *pinfo;
-		int			nparts = subpart->nparts;
-		int		   *subnodeindex;
-		int		   *subpartindex;
-		List	   *partprunequal;
+		int			nparts = rel->nparts;
+		int		   *subplan_indexes;
+		int		   *parent_indexes;
+		List	   *translated_prunequal;
 		bool		constfalse;
 
-		rte = root->simple_rte_array[subpart->relid];
-
+		rte = root->simple_rte_array[rel->relid];
 		pinfo = makeNode(PartitionPruneInfo);
 		pinfo->reloid = rte->relid;
 
 		/*
 		 * The first item in the list is the target partitioned relation.  The
-		 * quals belong to this relation, so require no translation.
+		 * quals belong to this relation, so require no translation.  Also,
+		 * save the first rel as the root parent for all subsequent rels in
+		 * partitioned_rels.
 		 */
 		if (i == 0)
 		{
-			parentpart = subpart;
-			partprunequal = prunequal;
+			parent = rel;
+			translated_prunequal = prunequal;
 		}
 		else
-		{
 			/*
 			 * For sub-partitioned tables the columns may not be in the same
 			 * order as the parent, so we must translate the prunequal to make
 			 * it compatible with this relation.
 			 */
-			partprunequal = (List *)
+			translated_prunequal = (List *)
 						adjust_appendrel_attrs_multilevel(root,
 														  (Node *) prunequal,
-														  subpart->relids,
-														  parentpart->relids);
-		}
-
-		pinfo->prunesteps = generate_partition_pruning_steps(subpart,
-															 partprunequal,
-															 &constfalse);
+														  rel->relids,
+														  parent->relids);
+		pinfo->pruning_steps =
+						generate_partition_pruning_steps(rel,
+														 translated_prunequal,
+														 &constfalse);
 
 		if (constfalse)
 		{
@@ -378,10 +360,11 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 			return NIL;
 		}
 
-		pinfo->allpartindexes = NULL;
+		pinfo->unpruned_parts = NULL;
 		pinfo->nparts = nparts;
-		pinfo->subnodeindex = subnodeindex = palloc(nparts * sizeof(int));
-		pinfo->subpartindex = subpartindex = palloc(nparts * sizeof(int));
+		pinfo->subplan_indexes = subplan_indexes =
+												palloc(nparts * sizeof(int));
+		pinfo->parent_indexes = parent_indexes = palloc(nparts * sizeof(int));
 		pinfo->extparams = NULL;
 		pinfo->execparams = NULL;
 
@@ -390,7 +373,7 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 		 * We'll not bother enabling run-time pruning if no params matched
 		 * the partition key at any level of partitioning.
 		 */
-		gotparam |= pull_partkey_params(pinfo, pinfo->prunesteps);
+		gotparam |= pull_partkey_params(pinfo, pinfo->pruning_steps);
 
 		/*
 		 * Loop over each partition of the partitioned rel and record the
@@ -400,21 +383,28 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 		 */
 		for (i = 0; i < nparts; i++)
 		{
-			RelOptInfo *partrel = subpart->part_rels[i];
-			int			subnodeidx = allsubnodeindex[partrel->relid];
-			int			subpartidx = allsubpartindex[partrel->relid];
-
-			subnodeindex[i] = subnodeidx;
-			subpartindex[i] = subpartidx;
+			RelOptInfo *partrel = rel->part_rels[i];
+			int			subpath_index = all_subpath_indexes[partrel->relid];
+			int			parent_index = all_parent_indexes[partrel->relid];
 
 			/*
-			 * Record the indexes of all the partition indexes that we have
-			 * subnodes or subparts for.  This allows an optimization to skip
-			 * attempting any run-time pruning when no Params are found
-			 * matching the partition key at this level.
+			 * If this partition's path is in subpaths, add its offset to
+			 * subplan_indexes.  If it's not, -1 will be stored.
 			 */
-			if (subnodeidx >= 0 || subpartidx >= 0)
-				pinfo->allpartindexes = bms_add_member(pinfo->allpartindexes,
+			subplan_indexes[i] = subpath_index - 1;
+
+			/*
+			 * If this partition is itself a partitioned table, add its offset
+			 * to parent_indexes.  If it's not, -1 will be stored.
+			 */
+			parent_indexes[i] = parent_index - 1;
+
+			/*
+			 * Record the indexes of all the partitions that each either has
+			 * a subpath for or appears in partitioned_rels list.
+			 */
+			if (subpath_index > 0 || parent_index > 0)
+				pinfo->unpruned_parts = bms_add_member(pinfo->unpruned_parts,
 													   i);
 		}
 
@@ -422,8 +412,8 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 		i++;
 	}
 
-	pfree(allsubnodeindex);
-	pfree(allsubpartindex);
+	pfree(all_subpath_indexes);
+	pfree(all_parent_indexes);
 
 	if (gotparam)
 		return pinfolist;
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 140e0bdf1e..6afd513ee2 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -110,70 +110,72 @@ typedef struct PartitionTupleRouting
 } PartitionTupleRouting;
 
 /*-----------------------
- * PartitionedRelPruning - Encapsulates all information required to support
- * elimination of partitions in node types which support arbitrary Lists of
- * subplans.  Information stored here allows partprune.c's partition pruning
- * functions to be called and the return value of partition indexes translated
- * into the subpath indexes of node types such as Append, thus allowing us to
- * bypass certain subnodes when we have proofs that indicate that no tuple
- * matching the 'prunesteps' will be found within.
+ * PartitionPruningDispatch - Encapsulates all information required to perform
+ * partition pruning using the information provided in one of the node types
+ * used to scan partitioned tables, viz. Append and MergeAppend.  Information
+ * stored here allows us to call partprune.c's partition pruning functions and
+ * translate the returned partition indexes into those of the subplans
+ * subplans contained in the Append or MergeAppend or the index of another
+ * PartitionPruningDispatch if we need to perform pruning one level down.
  *
- * nparts						The number of partitions which belong to this
- *								partitioned relation. Also defines the size of
- *								the 'subnodeindex' and 'subpartprune' arrays.
- * subnodeindex					An array of nparts containing the subnode
- *								index which matches this partition index, or
- *								-1 if there is no match.
- * subpartprune					An array of nparts containing the
- *								PartitionedRelPruning details this partition
- *								index for sub-partitioned tables.
- * allpartindexes				A Bitmapset of the partition index that we have
- *								subnodes mapped for.
- *								belong to this partition.
- * context						Contains the context details required to call
- *								the partition pruning code.
- * prunesteps					Contains list of PartitionPruneStep used to
- *								perform the actual pruning.
+ * context					Contains the context details required to call
+ *							the partition pruning code.
+ * pruning_steps			Contains list of PartitionPruneStep used to
+ *							perform the actual pruning.
+ * extparams				IDs of external Params
+ * allparams				IDs of both exeternal and executor Params
+ * unpruned_parts			Indexes of partitions selected after executing
+ *							pruning_steps
+ * subplan_indexes			An array containing one value for each partition,
+ *							which, if it's >= 0, is the index of its subplan
+ *							in the correponding Append or MergeAppend node
+ *							or -1 if the partition has been pruned or is not
+ *							a leaf partition
+ * parent_indexes			An array containing one value for each partition,
+ *							which, if it's >= 0, is the index of its
+ *							PartitionPruningDispatchData to perform further
+ *							pruning with (that is, pruning of the partitions
+ *							of the next level) or -1 if the partition has
+ *							been pruned or is a leaf partitions.
  *-----------------------
  */
-typedef struct PartitionedRelPruning
+typedef struct PartitionPruningDispatchData
 {
-	int			nparts;
-	int		   *subnodeindex;
-	struct PartitionedRelPruning **subpartprune;
-	Bitmapset  *allpartindexes;
 	PartitionPruneContext context;
-	List	   *prunesteps;
+	List	   *pruning_steps;
 	Bitmapset  *extparams;
 	Bitmapset  *allparams;
-} PartitionedRelPruning;
+	Bitmapset  *unpruned_parts;
+	int		   *subplan_indexes;
+	int		   *parent_indexes;
+} PartitionPruningDispatchData;
+
+typedef struct PartitionPruningDispatchData *PartitionPruningDispatch;
 
 /*-----------------------
- * PartitionPruning - Encapsulates a hierarchy of PartitionedRelPruning
- * structs and also stores all Param IDs which were found to match the
- * partition keys of each partition.  This struct can be attached to node
- * types which support arbitrary Lists of subnodes containing partitions to
- * allow subnodes to be eliminated due to the clauses being unable to match
- * to any tuple that the subnode could possibly produce.
+ * PartitionPruning - Analogous to PartitionTupleRouting, this encapsulates
+ * the information needed to perform partition pruning for the partitioned
+ * tables in the tree
  *
- * partrelpruning		Array of PartitionedRelPruning for the node's target
- *						partitioned relation. First element contains the
- *						details for the target partitioned table.
- * npartrelpruning		Number of items in partrelpruning array.
- * prune_context		A memory context which can be used to call the query
- *						planner's partition prune functions.
- * extparams			All PARAM_EXTERN Param IDs which were found to match a
- *						partition key in each of the contained
- *						PartitionedRelPruning structs.
- * execparams			As above but for PARAM_EXEC.
- * allparams			Union of extparams and execparams, saved to avoid
- *						recalculation.
+ * partition_dispatch_info	Array of PartitionPruningDispatch objects with
+ *							one entry for each partitioned table in the
+ *							partition tree.
+ * num_dispatch				number of partitioned tables in the partition
+ *							tree (= length of partition_dispatch_info[])
+ * prune_context			A memory context which can be used to call the
+ *							planner's partition prune functions.
+ * extparams				All PARAM_EXTERN Param IDs which were found to
+ *							match a partition key in each of the contained
+ *							PartitionedRelPruning structs.
+ * execparams				As above but for PARAM_EXEC.
+ * allparams				Union of extparams and execparams, saved to avoid
+ *							recalculation.
  *-----------------------
  */
 typedef struct PartitionPruning
 {
-	PartitionedRelPruning *partrelpruning;
-	int npartrelpruning;
+	PartitionPruningDispatch *partition_dispatch_info;
+	int		num_dispatch;
 	MemoryContext prune_context;
 	Bitmapset *extparams;
 	Bitmapset *execparams;
@@ -201,8 +203,8 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
 extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
 						  List *partitionpruneinfo);
-extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
-extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *pprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *pprune,
 								int nsubplans);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index f357473c6a..750784bbd8 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1580,26 +1580,44 @@ typedef struct PartitionPruneStepCombine
 } PartitionPruneStepCombine;
 
 /*----------
- * PartitionPruneInfo - Details required to allow the executor to prune
- * partitions.
+ * PartitionPruneInfo - Details about partition pruning that planner passes
+ * to the executor through the Plan node types that are used for partitioned
+ * tables
  *
- * Here we store mapping details to allow translation of a partitioned table's
- * index into subnode indexes for node types which support arbitrary numbers
- * of sub nodes, such as Append.
+ * Here we store three pieces of information:
+ *
+ *	1. Partition pruning steps which contain references to Params that can
+ *	   only be evaluated during execution
+ *
+ *	2. Mapping details to allow translation of partition indexes (as stored in
+ *	   the table's partition descriptor to the indexes of the corresponding
+ *	   partition's subplan or if the partition is itself a partitioned table,
+ *	   the index of a struct that's in turn can be used to perform partition
+ *	   pruning for its own partition.
+ *
+ *	3. Param IDs for both both Param types.
+ *
+ * This closely resembles the PartitionDispatchData in execPartition.h, but
+ * since this is stored in a plan it's defined as a Node type.
  *----------
  */
 typedef struct PartitionPruneInfo
 {
 	NodeTag		type;
-	Oid			reloid;		/* Oid of partition rel */
-	List	   *prunesteps;	/* List of PartitionPruneStep */
-	Bitmapset  *allpartindexes;	/* All part index we have subnodes for at this
-								 * level */
-	int			nparts;		/* length of the following arrays */
-	int		   *subnodeindex;	/* subnode index indexed by partition id */
-	int		   *subpartindex;	/* subpart index indexed by partition id */
-	Bitmapset  *extparams;		/* All external ParamIDs seen in prunesteps */
-	Bitmapset  *execparams;		/* All exec ParamIDs seen in prunesteps */
+	Oid			reloid;			/* OID of the table this node is for */
+	List	   *pruning_steps;	/* List of PartitionPruneStep */
+	Bitmapset  *unpruned_parts;	/* Indexes of all unpruned partitions */
+
+	int			nparts;			/* Total number of table's partitions */
+	int		   *subplan_indexes;	/* Map from partition indexes to
+									 * subplan indexes, for leaf partitions */
+	int		   *parent_indexes;		/* Map from partition indexes to
+									 * index of the pruning info struct, for
+									 * partitioned partitions */
+	Bitmapset  *extparams;		/* All external ParamIDs seen in
+								 * pruning_steps */
+	Bitmapset  *execparams;		/* All executor ParamIDs seen in
+								 * pruning_steps */
 } PartitionPruneInfo;
 
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index b7352d150c..65c0c23560 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -20,8 +20,8 @@ extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
-extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
-						 List *resultRelations, List *subpaths,
+extern List *make_partition_pruneinfo(PlannerInfo *root,
+						 List *partitioned_rels, List *subpaths,
 						 List *prunequal);
 
 #endif							/* PARTPRUNE_H */
#112David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#111)
Re: [HACKERS] Runtime Partition Pruning

Hi Amit,

Thanks for having a look at this.

On 6 April 2018 at 00:54, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

I looked at it and here are my thoughts on it after having for the first
time looked very closely at it.

* Regarding PartitionPruneInfo:

I think the names of the fields could be improved -- like pruning_steps
instead prunesteps, unpruned_parts instead of allpartindexs. The latter
is even a bit misleading because it doesn't in fact contain *all*
partition indexes, only those that are unpruned, because each either has a
subpath or it appears in (unpruned) partitioned_rels list. Also, I didn't
like the name subnodeindex and subpartindex very much. subplan_indexes
and parent_indexes would sound more informative to me.

Seems mostly fair. I'm not a fan of using the term "unpruned" though.
I'll have a think. The "all" is meant in terms of what exists as
subnodes.

subplan_indexes and parent_indexes seem like better names, I agree.

* make_partition_pruneinfo has a parameter resultRelations that's not used
anywhere

It gets used in 0005.

I guess I could only add it in 0005, but make_partition_pruneinfo is
only used in 0003, so you could say the same about that entire
function.

Do you think I should delay adding that parameter until the 0005 patch?

* In make_partition_pruneinfo()

allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);

I think these arrays need to have root->simple_rel_array_size + 1
elements, because they're subscripted using RT indexes which start at 1.

RT indexes are always 1-based. See setup_simple_rel_arrays. It already
sets the array size to list_length(rtable) + 1.

* Also in make_partition_pruneinfo()

/* Initialize to -1 to indicate the rel was not found */
for (i = 0; i < root->simple_rel_array_size; i++)
{
allsubnodeindex[i] = -1;
allsubpartindex[i] = -1;
}

Maybe, allocate the arrays above mentioned using palloc0 and don't do this
initialization. Instead make the indexes that are stored in these start
with 1 and consider 0 as invalid entries.

0 is a valid subplan index. It is possible to make this happen, but
I'd need to subtract 1 everywhere I used the map. That does not seem
very nice. Seems more likely to result in bugs where we might forget
to do the - 1.

Did you want this because you'd rather have the palloc0() than the for
loop setting the array elements to -1? Or is there another reason?

* Regarding the code added in execPartition.c and execPartition.h:

I wondered why PartitionedRelPruning is named the way it is. I saw many
parallels with PartitionDispatchData both in terms of the main thing it
consists of, that is, the map that translates partition indexes as in
partition descriptor to that of subplans or of some other executor
structure. Also, I imagine you tried to mimic PartitionTupleRouting with
PartitionPruning but not throughout. For example, tuple routing struct
pointer variables are throughout called proute, whereas PartitionPruning
ones are called partprune instead of, say, pprune. Consistency would
help, imho.

Yes, I saw similarities and did end up moving all the code into
execPartition a while back.

I'll look into this renaming.

* Instead of nesting PartitionedRelPruning inside another, just store them
in a global flat array in the PartitionPruning, like PartitionTupleRouting
does for PartitionDispatch of individual partitioned tables in the tree.

typedef struct PartitionedRelPruning
{
int nparts;
int *subnodeindex;
struct PartitionedRelPruning **subpartprune;

There is a flat array in PartitionPruning. subpartprune contains
pointers into that array. I want to have this pointer array so I can
directly reference the flat array in PartitionPruning.

Maybe I've misunderstood what you mean here.

* I don't see why there needs to be nparts in the above, because it
already has a PartitionPruneContext member which has that information.

Good point. I'll remove that.

In fact, I made most of changes myself while going through the code.
Please see attached the delta patch. It also tweaks quite a few other
things including various comments. I think parts of it apply to 0001,
0003, and 0004 patches. See if this looks good to you.

Thanks. I'll look.

It's late over this side now, so will look tomorrow.

Thanks again for reviewing this.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#113Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#112)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

First of all: Solid patch set with good documentation.

On 04/05/2018 09:41 AM, David Rowley wrote:

Seems mostly fair. I'm not a fan of using the term "unpruned" though.
I'll have a think. The "all" is meant in terms of what exists as
subnodes.

'included_parts' / 'excluded_parts' probably isn't better...

subplan_indexes and parent_indexes seem like better names, I agree.

More clear.

* Also in make_partition_pruneinfo()

/* Initialize to -1 to indicate the rel was not found */
for (i = 0; i < root->simple_rel_array_size; i++)
{
allsubnodeindex[i] = -1;
allsubpartindex[i] = -1;
}

Maybe, allocate the arrays above mentioned using palloc0 and don't do this
initialization. Instead make the indexes that are stored in these start
with 1 and consider 0 as invalid entries.

0 is a valid subplan index. It is possible to make this happen, but
I'd need to subtract 1 everywhere I used the map. That does not seem
very nice. Seems more likely to result in bugs where we might forget
to do the - 1.

Did you want this because you'd rather have the palloc0() than the for
loop setting the array elements to -1? Or is there another reason?

I think that doing palloc0 would be confusing; -1 is more clear,
especially since it is used with 'allpartindexes' which is a Bitmapset.

Doing the variable renames as Amit suggests would be good.

I ran some tests (v50_v20) (make check-world passes), w/ and w/o
choose_custom_plan() being false, and seeing good performance results
without running into issues.

Maybe 0005 should be expanded in partition_prune.sql with the supported
cases to make those more clear.

Thanks !

Best regards,
Jesper

#114Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#112)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/04/05 22:41, David Rowley wrote:

* make_partition_pruneinfo has a parameter resultRelations that's not used
anywhere

It gets used in 0005.

I guess I could only add it in 0005, but make_partition_pruneinfo is
only used in 0003, so you could say the same about that entire
function.

Do you think I should delay adding that parameter until the 0005 patch?

Yes, I think.

* In make_partition_pruneinfo()

allsubnodeindex = palloc(sizeof(int) * root->simple_rel_array_size);
allsubpartindex = palloc(sizeof(int) * root->simple_rel_array_size);

I think these arrays need to have root->simple_rel_array_size + 1
elements, because they're subscripted using RT indexes which start at 1.

RT indexes are always 1-based. See setup_simple_rel_arrays. It already
sets the array size to list_length(rtable) + 1.

Oh, I missed that simple_rel_array_size itself is set to consider 1-based
RT indexes.

relnode.c:73
root->simple_rel_array_size = list_length(root->parse->rtable) + 1;

* Also in make_partition_pruneinfo()

/* Initialize to -1 to indicate the rel was not found */
for (i = 0; i < root->simple_rel_array_size; i++)
{
allsubnodeindex[i] = -1;
allsubpartindex[i] = -1;
}

Maybe, allocate the arrays above mentioned using palloc0 and don't do this
initialization. Instead make the indexes that are stored in these start
with 1 and consider 0 as invalid entries.

0 is a valid subplan index. It is possible to make this happen, but
I'd need to subtract 1 everywhere I used the map. That does not seem
very nice. Seems more likely to result in bugs where we might forget
to do the - 1.

You can subtract 1 right here in make_partition_pruneinfo before setting
the values in PartitionPruneInfo's subplan_indexes and parent_indexes.
I'm only proposing to make make_partition_pruneinfo() a bit faster by not
looping over both the local map arrays setting them to -1.

IOW, I'm not saying that we emit PartitionPruneInfo nodes that contain
1-based indexes.

Did you want this because you'd rather have the palloc0() than the for
loop setting the array elements to -1? Or is there another reason?

Yeah, that's it.

* Instead of nesting PartitionedRelPruning inside another, just store them
in a global flat array in the PartitionPruning, like PartitionTupleRouting
does for PartitionDispatch of individual partitioned tables in the tree.

typedef struct PartitionedRelPruning
{
int nparts;
int *subnodeindex;
struct PartitionedRelPruning **subpartprune;

There is a flat array in PartitionPruning. subpartprune contains
pointers into that array. I want to have this pointer array so I can
directly reference the flat array in PartitionPruning.

Maybe I've misunderstood what you mean here.

I think we can save some space here by not having the pointers stored
here. Instead of passing the pointer itself in the recursive calls to
find_subplans_for_extparams_recurse, et al, just pass the entire array and
an offset to use for the given recursive invocation.

If you look at ExecFindPartition used for tuple routing, you may see that
there no recursion at all. Similarly find_subplans_for_extparams_recurse,
et al might be able to avoid recursion if similar tricks are used.

Finally about having two different functions for different sets of Params:
can't we have just named find_subplans_for_params_recurse() and use the
appropriate one based on the value of some parameter? I can't help but
notice the duplication of code.

Thanks,
Amit

#115David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#114)
Re: [HACKERS] Runtime Partition Pruning

(sending my reply in parts for concurrency)

On 6 April 2018 at 14:39, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

I think we can save some space here by not having the pointers stored
here. Instead of passing the pointer itself in the recursive calls to
find_subplans_for_extparams_recurse, et al, just pass the entire array and
an offset to use for the given recursive invocation.

hmm, where would those offsets be stored? I don't want to have to do
any linear searching to determine the offset, which is why I just
stored the pointer to the flat array. It seems very efficient to me to
do this. Remember that this pruning can occur per tuple in cases like
parameterized nested loops.

Are you worried about memory consumption? It's one pointer per
partition. I imagine there's lots more allocated for DML on a
partitioned table as it needs to store maps to map attribute numbers.

Or are you thinking the saving of storing an array of 32-bit int
values is better than the array of probably 64-bit pointers? So
requires half the space?

If you look at ExecFindPartition used for tuple routing, you may see that
there no recursion at all. Similarly find_subplans_for_extparams_recurse,
et al might be able to avoid recursion if similar tricks are used.

That seems pretty different. That's looking for a single node in a
tree, so just is following a single path from the root, it never has
to go back up a level and look down any other paths.

What we need for the find_subplans_for_extparams_recurse is to find
all nodes in the entire tree which match the given clause. Doing this
without recursion would require some sort of stack so we can go back
up a level and search again down another branch. There are ways
around this without using recursion, sure, but I don't think any of
them will be quite as convenient and simple. The best I can think of
is to palloc some stack manually and use some depth_level to track
which element to use. An actual stack seems more simple. I can't
quite think of a good way to know in advance how many elements we'd
need to palloc.

Finally about having two different functions for different sets of Params:
can't we have just named find_subplans_for_params_recurse() and use the
appropriate one based on the value of some parameter? I can't help but
notice the duplication of code.

I had decided not to make this one function previously as I didn't
really want to add unnecessary branching in the code. After
implementing it, it does not look as bad as I thought.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#116Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#115)
Re: [HACKERS] Runtime Partition Pruning

Hi David,

On 2018/04/06 12:27, David Rowley wrote:

(sending my reply in parts for concurrency)

On 6 April 2018 at 14:39, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

I think we can save some space here by not having the pointers stored
here. Instead of passing the pointer itself in the recursive calls to
find_subplans_for_extparams_recurse, et al, just pass the entire array and
an offset to use for the given recursive invocation.

hmm, where would those offsets be stored? I don't want to have to do
any linear searching to determine the offset, which is why I just
stored the pointer to the flat array. It seems very efficient to me to
do this. Remember that this pruning can occur per tuple in cases like
parameterized nested loops.

Are you worried about memory consumption? It's one pointer per
partition. I imagine there's lots more allocated for DML on a
partitioned table as it needs to store maps to map attribute numbers.

Or are you thinking the saving of storing an array of 32-bit int
values is better than the array of probably 64-bit pointers? So
requires half the space?

Yeah, just copy it from the PartitionPruneInfo like you're doing for
subnodeindex:

memcpy(partrelprune->subpartindex, pinfo->subpartindex,
sizeof(int) * pinfo->nparts);

Instead I see ExecSetupPartitionPruning is now doing this:

/*
* Setup the PartitionedRelPruning's subpartprune so that we can
* quickly find sub-PartitionedRelPruning details for any
* sub-partitioned tables that this partitioned table contains.
* We need to be able to find these quickly during our recursive
* search to find all matching subnodes.
*/
for (j = 0; j < pinfo->nparts; j++)
{
int subpartidx = pinfo->subpartindex[j];

Assert(subpartidx < list_length(partitionpruneinfo));

if (subpartidx >= 0)
partrelprune->subpartprune[j] = &partrelprunes[subpartidx];
else
partrelprune->subpartprune[j] = NULL;
}

With that in place, pass the index/offset instead of the pointer to the
next recursive invocation of find_subplans_*, along with the array
containing all PartitionedRelPruning's.

So, where you have in each of find_subplans_*:

if (partrelprune->subnodeindex[i] >= 0)
*validsubplans = bms_add_member(*validsubplans,
partrelprune->subnodeindex[i]);
else if (partrelprune->subpartprune[i] != NULL)
find_subplans_for_allparams_recurse(partrelprune->subpartprune[i],
validsubplans);

I'm proposing that you do:

if (partrelprune->subnodeindex[i] >= 0)
*validsubplans = bms_add_member(*validsubplans,
partrelprune->subnodeindex[i]);
else if (partrelprune->subpartindex[i] >= 0)
find_subplans_for_allparams_recurse(all_partrelprunes,
partrelprune->subpartindex[i],
validsubplans);

And at the top of each of find_subplans_*:

ParitionedRelPruning *partrelprune = all_partrelprunes[offset];

where the signature is:

static void
find_subplans_for_allparams_recurse(
PartitionRelPruning **all_partrelprune,
int offset,
Bitmapset **validsubplans)

The all_partrelprune above refers to the flat array in PartitionPruning.
On the first call from ExecFindMatchingSubPlans, et al, you'd pass 0 for
offset to start pruning with the root parent's PartitionedRelPruning. All
the values contained in subnodeindex and subpartindex are indexes into the
global array (whole-tree that is) anyway and that fact would be more
apparent if we use this code structure, imho.

If you look at ExecFindPartition used for tuple routing, you may see that
there no recursion at all. Similarly find_subplans_for_extparams_recurse,
et al might be able to avoid recursion if similar tricks are used.

That seems pretty different. That's looking for a single node in a
tree, so just is following a single path from the root, it never has
to go back up a level and look down any other paths.

What we need for the find_subplans_for_extparams_recurse is to find
all nodes in the entire tree which match the given clause. Doing this
without recursion would require some sort of stack so we can go back
up a level and search again down another branch. There are ways
around this without using recursion, sure, but I don't think any of
them will be quite as convenient and simple. The best I can think of
is to palloc some stack manually and use some depth_level to track
which element to use. An actual stack seems more simple. I can't
quite think of a good way to know in advance how many elements we'd
need to palloc.

Hmm, yeah. I just remembered that I had to give up suggesting this a
while back on this thread. So, okay, you don't need to do anything about
this.

Finally about having two different functions for different sets of Params:
can't we have just named find_subplans_for_params_recurse() and use the
appropriate one based on the value of some parameter? I can't help but
notice the duplication of code.

I had decided not to make this one function previously as I didn't
really want to add unnecessary branching in the code. After
implementing it, it does not look as bad as I thought.

You could at the top of, say, find_subplans_for_params_recurse(..., bool
extparam):

Bitmapset *params = extparam
? partrelprune->extparams
: partrelprune->allparams;

ISTT, find_subplans_for_extparams_recurse and
find_subplans_for_allparams_recurse contain the exact same code except
these two lines in the two functions, respectively:

if (!bms_is_empty(partrelprune->extparams))
{
context->safeparams = partrelprune->extparams;

if (!bms_is_empty(partrelprune->allparams))
{
context->safeparams = partrelprune->allparams;

I didn't suggest the same for say ExecFindInitialMatchingSubPlans and
ExecFindMatchingSubPlans because they seem to be at least a bit different
in their missions. IIUC, the former has to do index value adjustment
(those in subnodeindex and subpartindex) after having discovered a new set
of matching partitions in the tree after pruning with external params at
Append/MergeAppend startup and those params won't change after that point.

Thanks,
Amit

#117David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#111)
5 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 6 April 2018 at 00:54, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:

* Regarding PartitionPruneInfo:

I think the names of the fields could be improved -- like pruning_steps
instead prunesteps, unpruned_parts instead of allpartindexs. The latter
is even a bit misleading because it doesn't in fact contain *all*
partition indexes, only those that are unpruned, because each either has a
subpath or it appears in (unpruned) partitioned_rels list. Also, I didn't
like the name subnodeindex and subpartindex very much. subplan_indexes
and parent_indexes would sound more informative to me.

I've done a load of renaming. I went with subnode_map and subpart_map
instead of _indexes. I thought this was better. Let me know if you
disagree.

* make_partition_pruneinfo has a parameter resultRelations that's not used
anywhere

I've taken this out of 0001 and it is now introduced again in 0005,
where it's used.

* Also in make_partition_pruneinfo()

/* Initialize to -1 to indicate the rel was not found */
for (i = 0; i < root->simple_rel_array_size; i++)
{
allsubnodeindex[i] = -1;
allsubpartindex[i] = -1;
}

Maybe, allocate the arrays above mentioned using palloc0 and don't do this
initialization. Instead make the indexes that are stored in these start
with 1 and consider 0 as invalid entries.

I've made this change.

* Regarding the code added in execPartition.c and execPartition.h:

I wondered why PartitionedRelPruning is named the way it is. I saw many
parallels with PartitionDispatchData both in terms of the main thing it
consists of, that is, the map that translates partition indexes as in
partition descriptor to that of subplans or of some other executor
structure. Also, I imagine you tried to mimic PartitionTupleRouting with
PartitionPruning but not throughout. For example, tuple routing struct
pointer variables are throughout called proute, whereas PartitionPruning
ones are called partprune instead of, say, pprune. Consistency would
help, imho.

I've made a series of changes here too, but didn't use the word
"Dispatch" anywhere. I'm not really sure what the origin of this word
is. To me, it means to send something somewhere, which I thought might
be why we see it tuple routing, since the tuple is being "dispatched"
to the correct partition.

We're not dispatching anything anywhere in partition pruning, so don't
really think the term can be used here. Although, if you see some
other reason for using that word, please explain.

* Instead of nesting PartitionedRelPruning inside another, just store them
in a global flat array in the PartitionPruning, like PartitionTupleRouting
does for PartitionDispatch of individual partitioned tables in the tree.

typedef struct PartitionedRelPruning
{
int nparts;
int *subnodeindex;
struct PartitionedRelPruning **subpartprune;

I've kept the same subpart_map from the PartitionPruneInfo. I actually
ended up using that one without performing a memcpy() on it, since
we're not changing it anywhere. That might or might not be a good idea
since way might day think we can change it which would alter the
plan's copy, but on the other hand, there's a little performance boost
in not performing a copy.

* I don't see why there needs to be nparts in the above, because it
already has a PartitionPruneContext member which has that information.

I've removed that field.

I've also been doing a bit of work on the 0005 patch:

1. It now properly supports skipping subplans when exec params can
eliminate certain partitions. Previously I'd only coded it to work
with external params.
2. Fixed bug where statement level triggers would not fire when all
partitions were pruned.
3. Added tests

The patch is still based on v50 of the runtime pruning patch [1]/messages/by-id/77a518ac-e4a0-4cd1-9988-e5d754a6501f@lab.ntt.co.jp

[1]: /messages/by-id/77a518ac-e4a0-4cd1-9988-e5d754a6501f@lab.ntt.co.jp

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v21-0001-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v21-0001-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From 741db5b7ae09d2ad936fda07c1311eaf59b948cb Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:17:31 +1200
Subject: [PATCH v21 1/5] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/catalog/partition.c        |  24 ++
 src/backend/commands/explain.c         |  51 ++--
 src/backend/executor/execPartition.c   | 423 +++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c          |  20 ++
 src/backend/nodes/outfuncs.c           |  27 +++
 src/backend/nodes/readfuncs.c          |  19 ++
 src/backend/optimizer/util/partprune.c | 234 ++++++++++++++++++
 src/include/catalog/partition.h        |  13 +
 src/include/executor/execPartition.h   |  75 ++++++
 src/include/nodes/nodes.h              |   1 +
 src/include/nodes/primnodes.h          |  23 ++
 src/include/optimizer/partprune.h      |   3 +
 src/tools/pgindent/typedefs.list       |   3 +
 13 files changed, 898 insertions(+), 18 deletions(-)

diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c
index 73631ca0e7..7fa20abd33 100644
--- a/src/backend/catalog/partition.c
+++ b/src/backend/catalog/partition.c
@@ -1937,6 +1937,30 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+
+			/*
+			 * 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;
+				bool		isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+												   context->planstate->ps_ExprContext,
+												   &isNull);
+
+				if (isNull)
+					return false;
+
+				return true;
+
+			}
 		default:
 			break;
 	}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 79f639d5e2..549622da93 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1811,28 +1811,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3173,18 +3178,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that was
+	 * specified in the plan means that some subnodes have been ignored per
+	 * instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index ad532773a3..6cfd026474 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,6 +39,10 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_params_recurse(PartitionPruning *pprune,
+								 PartitionRelPruning *partrelprune,
+								 bool allparams,
+								 Bitmapset **validsubplans);
 
 
 /*
@@ -1272,3 +1276,422 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all.  This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruning:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruning details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruning
+ *		Setup the required data structure which is required for calling
+ *		ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These contexts can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionRelPruning *partrelprunes;
+	PartitionPruning *pprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	pprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionRelPruning *)
+		palloc(sizeof(PartitionRelPruning) *
+			   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	pprune->partrelpruning = partrelprunes;
+	pprune->npartrelpruning = list_length(partitionpruneinfo);
+	pprune->extparams = NULL;
+	pprune->execparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	pprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+												  "Partition Prune",
+												  ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc partdesc;
+		Relation	rel;
+		PartitionKey partkey;
+		int			partnatts;
+
+		partrelprune->present_parts = bms_copy(pinfo->present_parts);
+		partrelprune->subnode_map = palloc(sizeof(int) * pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(partrelprune->subnode_map, pinfo->subnode_map,
+			   sizeof(int) * pinfo->nparts);
+
+		/* We can use the subpart_map verbatim, since we never modify it */
+		partrelprune->subpart_map = pinfo->subpart_map;
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL; /* empty for now */
+
+		partrelprune->pruning_steps = pinfo->pruning_steps;
+
+		partrelprune->extparams = bms_copy(pinfo->extparams);
+		partrelprune->allparams = bms_union(pinfo->extparams,
+											pinfo->execparams);
+
+		/*
+		 * Accumulate the paramids which match the partitioned keys of all
+		 * partitioned tables.
+		 */
+		pprune->extparams = bms_add_members(pprune->extparams,
+											pinfo->extparams);
+
+		pprune->execparams = bms_add_members(pprune->execparams,
+											 pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the paramids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	pprune->allparams = bms_union(pprune->extparams,
+								  pprune->execparams);
+
+	return pprune;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'pprune'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so only once during their
+ * init plan.  The caller must only initialize the subnodes which are returned
+ * by this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'pprune' has any extparams.
+ *
+ * 'nsubnodes' must be passed as the total number of unpruned subnodes.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruning *pprune, int nsubnodes)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(pprune->extparams));
+
+	partrelprune = pprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
+
+	/* Determine which subnodes match the external params */
+	find_subplans_for_params_recurse(pprune, partrelprune, false, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(pprune->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been performed for external Params.
+	 */
+	bms_free(pprune->extparams);
+	pprune->extparams = NULL;
+
+	/*
+	 * If any subnodes were pruned, we must re-sequence the subnode indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subnodes which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubnodes)
+	{
+		int		   *new_subnode_indexes;
+		int			i;
+		int			newidx;
+
+		/*
+		 * First we must build an array which we can use to adjust the
+		 * existing subnode_map so that it contains the new subnode indexes.
+		 */
+		new_subnode_indexes = (int *) palloc(sizeof(int) * nsubnodes);
+		newidx = 0;
+		for (i = 0; i < nsubnodes; i++)
+		{
+			if (bms_is_member(i, result))
+				new_subnode_indexes[i] = newidx++;
+			else
+				new_subnode_indexes[i] = -1;	/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnode_map so
+		 * that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < pprune->npartrelpruning; i++)
+		{
+			PartitionRelPruning *partrelprune;
+			int			nparts;
+			int			j;
+
+			partrelprune = &pprune->partrelpruning[i];
+			nparts = partrelprune->context.nparts;
+
+			/*
+			 * We also need to reset the present_parts field so that it only
+			 * contains partition indexes that we actually still have subnodes
+			 * for.  It seems easier to build a fresh one, rather than trying
+			 * to update the existing one.
+			 */
+			bms_free(partrelprune->present_parts);
+			partrelprune->present_parts = NULL;
+
+			for (j = 0; j < nparts; j++)
+			{
+				int			oldidx = partrelprune->subnode_map[j];
+
+				/*
+				 * If this partition existed as a subnode then change the old
+				 * subnode index to the new subnode index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subnode list due to some subnodes being
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					partrelprune->subnode_map[j] = new_subnode_indexes[oldidx];
+
+					if (new_subnode_indexes[oldidx] >= 0)
+						partrelprune->present_parts =
+							bms_add_member(partrelprune->present_parts,
+										   j);
+				}
+			}
+		}
+
+		pfree(new_subnode_indexes);
+	}
+
+
+
+	return result;
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'pprune' for the current Param values.
+ *
+ * Here we utilize both external and exec Params for pruning.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *pprune)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	partrelprune = pprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
+
+	find_subplans_for_params_recurse(pprune, partrelprune, true, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(pprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_params_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans and
+ *		ExecFindInitialMatchingSubPlans
+ */
+static void
+find_subplans_for_params_recurse(PartitionPruning *pprune,
+								 PartitionRelPruning *partrelprune,
+								 bool allparams,
+								 Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset  *partset;
+	Bitmapset  *pruneparams;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * Use only external params unless we've been asked to also use exec
+	 * params too.
+	 */
+	if (allparams)
+		pruneparams = partrelprune->allparams;
+	else
+		pruneparams = partrelprune->extparams;
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(pruneparams))
+	{
+		context->safeparams = pruneparams;
+		partset = get_matching_partitions(context,
+										  partrelprune->pruning_steps);
+	}
+	else
+		partset = partrelprune->present_parts;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnode_map[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnode_map[i]);
+		else
+		{
+			int			partidx = partrelprune->subpart_map[i];
+
+			if (partidx != -1)
+				find_subplans_for_params_recurse(pprune,
+												 &pprune->partrelpruning[partidx],
+												 allparams, validsubplans);
+			else
+			{
+				/*
+				 * This could only happen if clauses used in planning where
+				 * more restrictive than those used here, or if the maps are
+				 * somehow corrupt.
+				 */
+				elog(ERROR, "partition missing from subplans");
+			}
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index b0fa556f71..be8ca35e93 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2167,6 +2167,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(pruning_steps);
+	COPY_BITMAPSET_FIELD(present_parts);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnode_map, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpart_map, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5103,6 +5120,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index e6793b4716..870613a5f5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1746,6 +1746,30 @@ _outOnConflictExpr(StringInfo str, const OnConflictExpr *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(pruning_steps);
+	WRITE_BITMAPSET_FIELD(present_parts);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnode_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnode_map[i]);
+
+	appendStringInfoString(str, " :subpart_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpart_map[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3977,6 +4001,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 25874074a0..8f1231dbfa 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1357,6 +1357,23 @@ _readPartitionPruneStepCombine(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(pruning_steps);
+	READ_BITMAPSET_FIELD(present_parts);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnode_map, local_node->nparts);
+	READ_INT_ARRAY(subpart_map, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2626,6 +2643,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 5b306193e1..390efdc7a8 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -27,6 +27,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/partprune.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -81,6 +82,7 @@ typedef struct GeneratePruningStepsContext
 	List   *steps;
 }			GeneratePruningStepsContext;
 
+static bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static List *generate_partition_pruning_steps_internal(RelOptInfo *rel,
 									  GeneratePruningStepsContext *context,
 									  List *clauses,
@@ -166,6 +168,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 		context.nparts = rel->nparts;
 		context.boundinfo = rel->boundinfo;
 
+		/* Not valid when being called from the planner */
+		context.planstate = NULL;
+		context.safeparams = NULL;
+
 		partindexes = get_matching_partitions(&context, pruning_steps);
 
 		/* Add selected partitions' RT indexes to result. */
@@ -230,8 +236,236 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 	return context.steps;
 }
 
+/*
+ * make_partition_pruneinfo
+ *		Builds List of PartitionPruneInfos, one for each 'partitioned_rels'.
+ *		These can be used in the executor to allow additional partition
+ *		pruning to take place.
+ *
+ * Here we generate partition pruning steps for 'prunequal' and also build a
+ * data stucture which allows mapping of partition indexes into 'subpaths'
+ * indexes.
+ *
+ * If no Params were found to match the partition key in any of the
+ * 'partitioned_rels', then we return NIL.  In such a case run-time partition
+ * pruning would be useless.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal)
+{
+	RelOptInfo *targetpart = NULL;
+	ListCell   *lc;
+	List	   *pinfolist = NIL;
+	int		   *relid_subnode_map;
+	int		   *relid_subpart_map;
+	int			i;
+	bool		gotparam = false;
+
+	/*
+	 * Allocate two arrays to store the 1-based indexes of the 'subpaths' and
+	 * 'partitioned_rels' by relid.
+	 */
+	relid_subnode_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+	relid_subpart_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+
+	i = 1;
+	foreach(lc, subpaths)
+	{
+		Path	   *path = (Path *) lfirst(lc);
+		RelOptInfo *pathrel = path->parent;
+
+		Assert(IS_SIMPLE_REL(pathrel));
+		Assert(pathrel->relid < root->simple_rel_array_size);
+
+		relid_subnode_map[pathrel->relid] = i++;
+	}
+
+	/* Likewise for the partition_rels */
+	i = 1;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		relid_subpart_map[rti] = i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		RangeTblEntry *rte;
+		Bitmapset  *present_parts;
+		int			nparts = subpart->nparts;
+		int		   *subnode_map;
+		int		   *subpart_map;
+		List	   *partprunequal;
+		List	   *pruning_steps;
+		bool		constfalse;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (!targetpart)
+		{
+			targetpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+				adjust_appendrel_attrs_multilevel(root,
+												  (Node *) prunequal,
+												  subpart->relids,
+												  targetpart->relids);
+		}
+
+		pruning_steps = generate_partition_pruning_steps(subpart,
+														 partprunequal,
+														 &constfalse);
+
+		if (constfalse)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered as the Param
+			 * values are unknown during planning.  Anyway, we'd better do
+			 * something sane here, so let's just disable run-time pruning.
+			 */
+			return NIL;
+		}
+
+		subnode_map = (int *) palloc(nparts * sizeof(int));
+		subpart_map = (int *) palloc(nparts * sizeof(int));
+		present_parts = NULL;
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present in
+		 * the subpaths List will be set to -1, and any sub-partitioned table
+		 * which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = relid_subnode_map[partrel->relid] - 1;
+			int			subpartidx = relid_subpart_map[partrel->relid] - 1;
+
+			subnode_map[i] = subnodeidx;
+			subpart_map[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				present_parts = bms_add_member(present_parts, i);
+		}
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+		pinfo->pruning_steps = pruning_steps;
+		pinfo->present_parts = present_parts;
+		pinfo->nparts = nparts;
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+		pinfo->subnode_map = subnode_map;
+		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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pruning_steps);
+
+		pinfolist = lappend(pinfolist, pinfo);
+	}
+
+	pfree(relid_subnode_map);
+	pfree(relid_subpart_map);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
+
 /* Module-local functions */
 
+/*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell   *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							 (int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
 /*
  * generate_partition_pruning_steps_internal
  *		Processes 'clauses' to generate partition pruning steps.
diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h
index b4b4844f20..9bd7d2077e 100644
--- a/src/include/catalog/partition.h
+++ b/src/include/catalog/partition.h
@@ -16,6 +16,7 @@
 #include "fmgr.h"
 #include "executor/tuptable.h"
 #include "nodes/execnodes.h"
+#include "nodes/relation.h"
 #include "parser/parse_node.h"
 #include "utils/rel.h"
 
@@ -113,6 +114,18 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow params
+	 * found matching the partition key to be evaulated.
+	 */
+	PlanState  *planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until after init plan.
+	 */
+	Bitmapset  *safeparams;
 } PartitionPruneContext;
 
 extern void RelationBuildPartitionDesc(Relation relation);
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 9f55f6409e..2d9db579f4 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "optimizer/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,75 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows the planner's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'pruning_steps' will be found within.
+ *
+ * subnode_map					An array containing the subnode index which
+ *								matches this partition index, or -1 if the
+ *								subnode has been pruned already.
+ * subpart_map					An array containing the offset into the
+ *								'partrelpruning' array in PartitionPruning, or
+ *								-1 if there is no such element in that array.
+ * present_parts				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * pruning_steps				Contains a list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ * extparams					Contains paramids of external params found
+ *								matching partition keys in 'pruning_steps'.
+ * allparams					As 'extparams' but also including exec params.
+ *-----------------------
+ */
+typedef struct PartitionRelPruning
+{
+	int		   *subnode_map;
+	int		   *subpart_map;
+	Bitmapset  *present_parts;
+	PartitionPruneContext context;
+	List	   *pruning_steps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionRelPruning
+ * structs and also stores all paramids which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partrelpruning		Array of PartitionRelPruning for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * npartrelpruning		Number of items in 'partrelpruning' array.
+ * 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
+ *						partition key in each of the contained
+ *						PartitionRelPruning structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of 'extparams' and 'execparams', saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionRelPruning *partrelpruning;
+	int			npartrelpruning;
+	MemoryContext prune_context;
+	Bitmapset  *extparams;
+	Bitmapset  *execparams;
+	Bitmapset  *allparams;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -127,5 +197,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+								int nsubnodes);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 1ec8030d4b..940469e41a 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -196,6 +196,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index 965eb656a8..f2d03ca210 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1579,4 +1579,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;			/* Oid of partition rel */
+	List	   *pruning_steps;	/* List of PartitionPruneStep */
+	Bitmapset  *present_parts;	/* Indexes of all partitions which subnodes
+								 * are present for. */
+	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 */
+	Bitmapset  *extparams;		/* All external paramids seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec paramids seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index 1f2fe297a3..ff8d18847f 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -20,4 +20,7 @@ extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal);
+
 #endif							/* PARTPRUNE_H */
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index adde8eaee9..76ea85b733 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1602,12 +1602,15 @@ PartitionHashBound
 PartitionKey
 PartitionListValue
 PartitionPruneContext
+PartitionPruneInfo
 PartitionPruneStep
 PartitionPruneStepCombine
 PartitionPruneStepOp
+PartitionPruning
 PartitionRangeBound
 PartitionRangeDatum
 PartitionRangeDatumKind
+PartitionRelPruning
 PartitionScheme
 PartitionSpec
 PartitionTupleRouting
-- 
2.16.2.windows.1

v21-0002-Add-bms_prev_member-function.patchapplication/octet-stream; name=v21-0002-Add-bms_prev_member-function.patchDownload
From 37fe490d345123f36798a7c29a63834376c15d7b Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:17:58 +1200
Subject: [PATCH v21 2/5] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1088,6 +1110,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 	return -2;
 }
 
+/*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
 /*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.16.2.windows.1

v21-0003-Allow-unneeded-Append-subnodes-to-be-pruned-at-e.patchapplication/octet-stream; name=v21-0003-Allow-unneeded-Append-subnodes-to-be-pruned-at-e.patchDownload
From b100a0d7fad448ba432ae7e3b7bc3276afe9a487 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:23:16 +1200
Subject: [PATCH v21 3/5] Allow unneeded Append subnodes to be pruned at
 execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  256 +++++-
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   15 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1800 insertions(+), 86 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..6e21b4b708 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +101,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -112,55 +116,118 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+	appendstate->as_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *pprune;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		pprune = ExecSetupPartitionPruning(&appendstate->ps,
+										   node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(pprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune,
+															list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set a noop flag so that we never actually
+			 * bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(pprune->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->partition_pruning = pprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+											 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the appendplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
+
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
+
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +318,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +350,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +440,40 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/* Handle the case for when all subplans were pruned */
+	if (node->as_noopscan)
+		return false;
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -411,10 +502,33 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
 	else
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +581,25 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
+
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+			ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +665,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int			i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index be8ca35e93..e6b4b5d740 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -248,6 +248,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index c3f1789ce2..db1470ce16 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3802,32 +3802,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3877,15 +3877,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 870613a5f5..69ab63d0bf 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -418,6 +418,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 8f1231dbfa..b80f7b5ade 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1675,6 +1675,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index c36a254ed6..25cfa5e349 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1607,7 +1607,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1649,8 +1649,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1698,7 +1698,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1761,7 +1761,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2027,7 +2027,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99d0736029..7f72bd0726 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1041,6 +1042,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1078,6 +1081,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1086,7 +1120,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5382,7 +5417,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5394,7 +5430,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 008492bad5..421dc79cc4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3920,7 +3920,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6852,8 +6853,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 67e47887fc..2ce4d4496d 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 416b3f9578..bd9442c22d 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3574,7 +3592,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ff63d179b2..b63c0c5329 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1114,8 +1114,15 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
+ *		noopscan			true if partition pruning proved that none of the
+ *							appendplans can contain a record to satisfy this
+ *							query.
  * ----------------
  */
 
@@ -1123,6 +1130,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruning;
 
 struct AppendState
 {
@@ -1132,7 +1140,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
+	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0a797f0a05..c3e5c2c79f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -256,6 +256,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 895bf6959d..4f65686d9b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 2d77b3edd4..0a2517125c 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1516,3 +1516,1138 @@ explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 (9 rows)
 
 drop table hp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index ad5177715c..eefbf32e93 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -274,3 +274,347 @@ explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and
 explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 
 drop table hp;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.16.2.windows.1

v21-0004-Allow-unneeded-MergeAppend-s-subnodes-to-be-prun.patchapplication/octet-stream; name=v21-0004-Allow-unneeded-MergeAppend-s-subnodes-to-be-prun.patchDownload
From 8799b735f6f7c791cd47ce1ba88e8f55cf426fad Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:29:29 +1200
Subject: [PATCH v21 4/5] Allow unneeded MergeAppend's subnodes to be pruned at
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   5 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 353 insertions(+), 21 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..31bc85b4b1 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,10 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -78,19 +81,89 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *pprune;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		pprune = ExecSetupPartitionPruning(&mergestate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(pprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune,
+															list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(pprune->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->partition_pruning = pprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +174,24 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +249,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +376,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index e6b4b5d740..57222d8f74 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -276,6 +276,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 69ab63d0bf..83e67627ad 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -450,6 +450,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index b80f7b5ade..1667881127 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1697,6 +1697,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 7f72bd0726..093ceaa867 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1144,6 +1144,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1229,8 +1231,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index b63c0c5329..1986abaa9c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1155,6 +1155,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1167,6 +1173,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c3e5c2c79f..a7dbd31466 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -280,6 +280,11 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 0a2517125c..792924fe0b 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2651,3 +2651,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index eefbf32e93..6f389bed1f 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -618,3 +618,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.16.2.windows.1

v21-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchapplication/octet-stream; name=v21-0005-Improve-planning-speed-of-partitioned-table-UPDA.patchDownload
From ea9cd0625c7c8741f8655f08750ea80b81fac018 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:30:19 +1200
Subject: [PATCH v21 5/5] Improve planning speed of partitioned table
 UPDATE/DELETEs

By making a call to grouping_planner for the complete parse of the query we
can make use of the faster partition pruning code used there.  This will
identify all partitions which could be pruned as IS_DUMMY_RELs, of which we
can skip performing each individual grouping_planner call inside
inheritance_planner.  This can improve planner performance significantly when
there are many partitions. There may be a slight slowdown when no partitions
could be pruned or when there are very few (1 or 2) partitions.  However it
seems better to optimize the case when partitions are pruned, rather than the
case where they're not, as those queries are less likely to be fast to
execute. The case for partitioned tables with just 1 or 2 leaf partitions does
not seem worth worrying about too much. The measured regression on 1 partition
was just 10% of overall planning time.

This commit also implements run-time partition pruning for UPDATE/DELETE.
---
 src/backend/commands/explain.c                |   4 +-
 src/backend/executor/execMerge.c              |   6 +-
 src/backend/executor/execPartition.c          |  18 +--
 src/backend/executor/nodeModifyTable.c        | 217 +++++++++++++++++++++-----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   1 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  36 ++++-
 src/backend/optimizer/plan/planner.c          |  59 +++++++
 src/backend/optimizer/util/partprune.c        |  40 +++--
 src/include/nodes/execnodes.h                 |  13 +-
 src/include/nodes/plannodes.h                 |   2 +
 src/include/optimizer/partprune.h             |   3 +-
 src/test/regress/expected/partition_prune.out | 110 +++++++++++++
 src/test/regress/sql/partition_prune.sql      |  34 ++++
 15 files changed, 472 insertions(+), 73 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 549622da93..12c933056d 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -3028,14 +3028,14 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 	/* Should we explicitly label target relations? */
 	labeltargets = (mtstate->mt_nplans > 1 ||
 					(mtstate->mt_nplans == 1 &&
-					 mtstate->resultRelInfo->ri_RangeTableIndex != node->nominalRelation));
+					 mtstate->resultRelInfos[0]->ri_RangeTableIndex != node->nominalRelation));
 
 	if (labeltargets)
 		ExplainOpenGroup("Target Tables", "Target Tables", false, es);
 
 	for (j = 0; j < mtstate->mt_nplans; j++)
 	{
-		ResultRelInfo *resultRelInfo = mtstate->resultRelInfo + j;
+		ResultRelInfo *resultRelInfo = mtstate->resultRelInfos[j];
 		FdwRoutine *fdwroutine = resultRelInfo->ri_FdwRoutine;
 
 		if (labeltargets)
diff --git a/src/backend/executor/execMerge.c b/src/backend/executor/execMerge.c
index d39ddd3034..7ff1bfc96b 100644
--- a/src/backend/executor/execMerge.c
+++ b/src/backend/executor/execMerge.c
@@ -203,8 +203,8 @@ ExecMergeMatched(ModifyTableState *mtstate, EState *estate,
 		if (resultRelInfo == NULL)
 		{
 			resultRelInfo = ExecInitPartitionInfo(mtstate,
-					mtstate->resultRelInfo,
-					proute, estate, leaf_part_index);
+												  mtstate->resultRelInfos[0],
+												  proute, estate, leaf_part_index);
 			Assert(resultRelInfo != NULL);
 		}
 	}
@@ -500,7 +500,7 @@ ExecMergeNotMatched(ModifyTableState *mtstate, EState *estate,
 	 * the currently active result relation, which corresponds to the root
 	 * of the partition tree.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 
 	/*
 	 * For INSERT actions, root relation's merge action is OK since the
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 6cfd026474..f76edd3523 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -101,7 +101,7 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
 
 	if (is_update)
 	{
-		update_rri = mtstate->resultRelInfo;
+		update_rri = mtstate->resultRelInfos[0];
 		num_update_rri = list_length(node->plans);
 		proute->subplan_partition_offsets =
 			palloc(num_update_rri * sizeof(int));
@@ -421,8 +421,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * In the case of INSERT on a partitioned table, there is only one
@@ -477,8 +477,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 		TupleTableSlot *slot;
 		ExprContext *econtext;
 		List	   *returningList;
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/* See the comment above for WCO lists. */
 		Assert((node->operation == CMD_INSERT &&
@@ -533,8 +533,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	if (node && node->onConflictAction != ONCONFLICT_NONE)
 	{
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		ExprContext *econtext = mtstate->ps.ps_ExprContext;
 		ListCell   *lc;
@@ -674,8 +674,8 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
 	{
 		TupleDesc	partrelDesc = RelationGetDescr(partrel);
 		TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
-		int			firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
-		Relation	firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
+		int			firstVarno = mtstate->resultRelInfos[0]->ri_RangeTableIndex;
+		Relation	firstResultRel = mtstate->resultRelInfos[0]->ri_RelationDesc;
 
 		/*
 		 * If the root parent and partition have the same tuple
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 0ebf37bd24..7f5d7a1bf4 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -54,6 +54,9 @@
 #include "utils/rel.h"
 #include "utils/tqual.h"
 
+/* Special values for mt_whichplan */
+#define WHICHPLAN_CHOOSE_PARTITIONS			-1
+#define WHICHPLAN_NO_MATCHING_PARTITIONS	-2
 
 static bool ExecOnConflictUpdate(ModifyTableState *mtstate,
 					 ResultRelInfo *resultRelInfo,
@@ -1214,12 +1217,12 @@ lreplace:;
 				map_index = resultRelInfo->ri_PartitionLeafIndex;
 				Assert(mtstate->rootResultRelInfo == NULL);
 				tupconv_map = TupConvMapForLeaf(proute,
-								mtstate->resultRelInfo,
-								map_index);
+												mtstate->resultRelInfos[0],
+												map_index);
 			}
 			else
 			{
-				map_index = resultRelInfo - mtstate->resultRelInfo;
+				map_index = mtstate->mt_whichplan;
 				Assert(map_index >= 0 && map_index < mtstate->mt_nplans);
 				tupconv_map = tupconv_map_for_subplan(mtstate, map_index);
 			}
@@ -1632,12 +1635,12 @@ static void
 fireBSTriggers(ModifyTableState *node)
 {
 	ModifyTable *plan = (ModifyTable *) node->ps.plan;
-	ResultRelInfo *resultRelInfo = node->resultRelInfo;
+	ResultRelInfo *resultRelInfo = node->resultRelInfos[0];
 
 	/*
 	 * If the node modifies a partitioned table, we must fire its triggers.
-	 * Note that in that case, node->resultRelInfo points to the first leaf
-	 * partition, not the root table.
+	 * Note that in that case, node->resultRelInfos[0] points to the first
+	 * leaf partition, not the root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		resultRelInfo = node->rootResultRelInfo;
@@ -1683,13 +1686,14 @@ static ResultRelInfo *
 getTargetResultRelInfo(ModifyTableState *node)
 {
 	/*
-	 * Note that if the node modifies a partitioned table, node->resultRelInfo
-	 * points to the first leaf partition, not the root table.
+	 * Note that if the node modifies a partitioned table,
+	 * node->resultRelInfos[0] points to the first leaf partition, not the
+	 * root table.
 	 */
 	if (node->rootResultRelInfo != NULL)
 		return node->rootResultRelInfo;
 	else
-		return node->resultRelInfo;
+		return node->resultRelInfos[0];
 }
 
 /*
@@ -1910,7 +1914,7 @@ static void
 ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 {
 	ResultRelInfo *targetRelInfo = getTargetResultRelInfo(mtstate);
-	ResultRelInfo *resultRelInfos = mtstate->resultRelInfo;
+	ResultRelInfo **resultRelInfos = mtstate->resultRelInfos;
 	TupleDesc	outdesc;
 	int			numResultRelInfos = mtstate->mt_nplans;
 	int			i;
@@ -1941,7 +1945,7 @@ ExecSetupChildParentMapForSubplan(ModifyTableState *mtstate)
 	for (i = 0; i < numResultRelInfos; ++i)
 	{
 		mtstate->mt_per_subplan_tupconv_maps[i] =
-			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i].ri_RelationDesc),
+			convert_tuples_by_name(RelationGetDescr(resultRelInfos[i]->ri_RelationDesc),
 								   outdesc,
 								   gettext_noop("could not convert row type"));
 	}
@@ -2079,8 +2083,47 @@ ExecModifyTable(PlanState *pstate)
 		node->fireBSTriggers = false;
 	}
 
+	if (node->mt_whichplan < 0)
+	{
+		/* Handle choosing the valid subpartitions */
+		if (node->mt_whichplan == WHICHPLAN_CHOOSE_PARTITIONS)
+		{
+			PartitionPruning *pprune = node->partition_pruning;
+
+			/* There should always be at least one */
+			Assert(node->mt_nplans > 0);
+
+			/*
+			 * When partition pruning is enabled and exec params match the
+			 * partition key then determine the minimum set of matching
+			 * subnodes.  Otherwise we match to all subnodes.
+			 */
+			if (pprune != NULL && !bms_is_empty(pprune->execparams))
+			{
+				node->mt_valid_subplans = ExecFindMatchingSubPlans(pprune);
+				node->mt_whichplan = bms_next_member(node->mt_valid_subplans, -1);
+
+				/* If no subplan matches these params then we're done */
+				if (node->mt_whichplan < 0)
+					goto done;
+			}
+			else
+			{
+				node->mt_valid_subplans = bms_add_range(NULL, 0,
+														node->mt_nplans - 1);
+				node->mt_whichplan = 0;
+			}
+		}
+
+		/* partition pruning determined that no partitions match */
+		else if (node->mt_whichplan == WHICHPLAN_NO_MATCHING_PARTITIONS)
+			goto done;
+		else
+			elog(ERROR, "invalid subplan index: %d", node->mt_whichplan);
+	}
+
 	/* Preload local variables */
-	resultRelInfo = node->resultRelInfo + node->mt_whichplan;
+	resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 	subplanstate = node->mt_plans[node->mt_whichplan];
 	junkfilter = resultRelInfo->ri_junkFilter;
 
@@ -2114,11 +2157,12 @@ ExecModifyTable(PlanState *pstate)
 		if (TupIsNull(planSlot))
 		{
 			/* advance to next subplan if any */
-			node->mt_whichplan++;
+			node->mt_whichplan = bms_next_member(node->mt_valid_subplans,
+												 node->mt_whichplan);
 
-			if (node->mt_whichplan < node->mt_nplans)
+			if (node->mt_whichplan >= 0)
 			{
-				resultRelInfo++;
+				resultRelInfo = node->resultRelInfos[node->mt_whichplan];
 				subplanstate = node->mt_plans[node->mt_whichplan];
 				junkfilter = resultRelInfo->ri_junkFilter;
 				estate->es_result_relation_info = resultRelInfo;
@@ -2285,6 +2329,8 @@ ExecModifyTable(PlanState *pstate)
 	/* Restore es_result_relation_info before exiting */
 	estate->es_result_relation_info = saved_resultRelInfo;
 
+done:
+
 	/*
 	 * We're done, but fire AFTER STATEMENT triggers before exiting.
 	 */
@@ -2309,9 +2355,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	ResultRelInfo *resultRelInfo;
 	Plan	   *subplan;
 	ListCell   *l;
-	int			i;
+	int			i,
+				j;
 	Relation	rel;
 	bool		update_tuple_routing_needed = node->partColsUpdated;
+	Bitmapset  *validsubplans;
 
 	/* check for unsupported flags */
 	Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -2327,9 +2375,73 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	mtstate->operation = operation;
 	mtstate->canSetTag = node->canSetTag;
 	mtstate->mt_done = false;
+	mtstate->mt_whichplan = WHICHPLAN_CHOOSE_PARTITIONS;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *pprune;
+
+		ExecAssignExprContext(estate, &mtstate->ps);
+
+		pprune = ExecSetupPartitionPruning(&mtstate->ps,
+										   node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away ModifyTable plans.
+		 */
+		if (!bms_is_empty(pprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune,
+															list_length(node->plans));
+
+			/*
+			 * If no plans match the given parameters then we must handle this
+			 * case in a special way.  The problem here is that code in
+			 * explain.c requires a ModifyTable to have at least one plan in
+			 * order for it to properly determine the Vars in that plan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan and set a special value for mt_whichplan to cause
+			 * it to never actually scan any subnodes.  We can't simply set
+			 * mt_done as we still need to fire any statement level triggers.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mtstate->mt_whichplan = WHICHPLAN_NO_MATCHING_PARTITIONS;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->plans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		mtstate->partition_pruning = pprune;
+	}
+	else
+	{
+		nplans = list_length(node->plans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * plans as valid, they must also all be initialized.
+		 */
+		validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		mtstate->partition_pruning = NULL;
+	}
+
 
 	mtstate->mt_plans = (PlanState **) palloc0(sizeof(PlanState *) * nplans);
-	mtstate->resultRelInfo = estate->es_result_relations + node->resultRelIndex;
+	mtstate->resultRelInfos = (ResultRelInfo **)
+		palloc(sizeof(ResultRelInfo *) * nplans);
 
 	/* If modifying a partitioned table, initialize the root table info */
 	if (node->rootResultRelIndex >= 0)
@@ -2353,8 +2465,6 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	 */
 	saved_resultRelInfo = estate->es_result_relation_info;
 
-	resultRelInfo = mtstate->resultRelInfo;
-
 	/*
 	 * mergeTargetRelation must be set if we're running MERGE and mustn't be
 	 * set if we're not.
@@ -2362,13 +2472,20 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	Assert(operation != CMD_MERGE || node->mergeTargetRelation > 0);
 	Assert(operation == CMD_MERGE || node->mergeTargetRelation == 0);
 
-	resultRelInfo->ri_mergeTargetRTI = node->mergeTargetRelation;
-
-	i = 0;
+	j = i = 0;
 	foreach(l, node->plans)
 	{
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
 		subplan = (Plan *) lfirst(l);
 
+		resultRelInfo = estate->es_result_relations + node->resultRelIndex + i;
+		mtstate->resultRelInfos[j] = resultRelInfo;
+
 		/* Initialize the usesFdwDirectModify flag */
 		resultRelInfo->ri_usesFdwDirectModify = bms_is_member(i,
 															  node->fdwDirectModifyPlans);
@@ -2405,7 +2522,7 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		/* Now init the plan for this result rel */
 		estate->es_result_relation_info = resultRelInfo;
-		mtstate->mt_plans[i] = ExecInitNode(subplan, estate, eflags);
+		mtstate->mt_plans[j] = ExecInitNode(subplan, estate, eflags);
 
 		/* Also let FDWs init themselves for foreign-table result rels */
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
@@ -2421,10 +2538,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 															 eflags);
 		}
 
-		resultRelInfo++;
 		i++;
+		j++;
 	}
 
+	mtstate->resultRelInfos[0]->ri_mergeTargetRTI = node->mergeTargetRelation;
+
 	estate->es_result_relation_info = saved_resultRelInfo;
 
 	/* Get the target relation */
@@ -2477,26 +2596,34 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 	/*
 	 * Initialize any WITH CHECK OPTION constraints if needed.
 	 */
-	resultRelInfo = mtstate->resultRelInfo;
-	i = 0;
+	j = i = 0;
 	foreach(l, node->withCheckOptionLists)
 	{
-		List	   *wcoList = (List *) lfirst(l);
+		List	   *wcoList;
 		List	   *wcoExprs = NIL;
 		ListCell   *ll;
 
+		if (!bms_is_member(i, validsubplans))
+		{
+			i++;
+			continue;
+		}
+
+		wcoList = (List *) lfirst(l);
+
 		foreach(ll, wcoList)
 		{
 			WithCheckOption *wco = (WithCheckOption *) lfirst(ll);
 			ExprState  *wcoExpr = ExecInitQual((List *) wco->qual,
-											   mtstate->mt_plans[i]);
+											   mtstate->mt_plans[j]);
 
 			wcoExprs = lappend(wcoExprs, wcoExpr);
 		}
-
+		resultRelInfo = mtstate->resultRelInfos[j];
 		resultRelInfo->ri_WithCheckOptions = wcoList;
 		resultRelInfo->ri_WithCheckOptionExprs = wcoExprs;
-		resultRelInfo++;
+
+		j++;
 		i++;
 	}
 
@@ -2526,15 +2653,25 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		/*
 		 * Build a projection for each result rel.
 		 */
-		resultRelInfo = mtstate->resultRelInfo;
+		j = i = 0;
 		foreach(l, node->returningLists)
 		{
-			List	   *rlist = (List *) lfirst(l);
+			List	   *rlist;
+
+			if (!bms_is_member(i, validsubplans))
+			{
+				i++;
+				continue;
+			}
 
+			rlist = (List *) lfirst(l);
+
+			resultRelInfo = mtstate->resultRelInfos[j];
 			resultRelInfo->ri_projectReturning =
 				ExecBuildProjectionInfo(rlist, econtext, slot, &mtstate->ps,
 										resultRelInfo->ri_RelationDesc->rd_att);
-			resultRelInfo++;
+			j++;
+			i++;
 		}
 	}
 	else
@@ -2545,12 +2682,10 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		 */
 		mtstate->ps.plan->targetlist = NIL;
 		ExecInitResultTupleSlotTL(estate, &mtstate->ps);
-
-		mtstate->ps.ps_ExprContext = NULL;
 	}
 
 	/* Set the list of arbiter indexes if needed for ON CONFLICT */
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (node->onConflictAction != ONCONFLICT_NONE)
 		resultRelInfo->ri_onConflictArbiterIndexes = node->arbiterIndexes;
 
@@ -2654,12 +2789,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 		}
 	}
 
-	resultRelInfo = mtstate->resultRelInfo;
+	resultRelInfo = mtstate->resultRelInfos[0];
 	if (mtstate->operation == CMD_MERGE)
 		ExecInitMerge(mtstate, estate, resultRelInfo);
 
 	/* select first subplan */
-	mtstate->mt_whichplan = 0;
 	subplan = (Plan *) linitial(node->plans);
 	EvalPlanQualSetPlan(&mtstate->mt_epqstate, subplan,
 						mtstate->mt_arowmarks[0]);
@@ -2709,11 +2843,11 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 
 		if (junk_filter_needed)
 		{
-			resultRelInfo = mtstate->resultRelInfo;
 			for (i = 0; i < nplans; i++)
 			{
 				JunkFilter *j;
 
+				resultRelInfo = mtstate->resultRelInfos[i];
 				subplan = mtstate->mt_plans[i]->plan;
 
 				if (operation == CMD_INSERT || operation == CMD_UPDATE)
@@ -2766,13 +2900,12 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
 				}
 
 				resultRelInfo->ri_junkFilter = j;
-				resultRelInfo++;
 			}
 		}
 		else
 		{
 			if (operation == CMD_INSERT)
-				ExecCheckPlanOutput(mtstate->resultRelInfo->ri_RelationDesc,
+				ExecCheckPlanOutput(mtstate->resultRelInfos[0]->ri_RelationDesc,
 									subplan->targetlist);
 		}
 	}
@@ -2819,7 +2952,7 @@ ExecEndModifyTable(ModifyTableState *node)
 	 */
 	for (i = 0; i < node->mt_nplans; i++)
 	{
-		ResultRelInfo *resultRelInfo = node->resultRelInfo + i;
+		ResultRelInfo *resultRelInfo = node->resultRelInfos[i];
 
 		if (!resultRelInfo->ri_usesFdwDirectModify &&
 			resultRelInfo->ri_FdwRoutine != NULL &&
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 57222d8f74..3cd8dcb4f1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -225,6 +225,7 @@ _copyModifyTable(const ModifyTable *from)
 	COPY_NODE_FIELD(exclRelTlist);
 	COPY_NODE_FIELD(mergeSourceTargetList);
 	COPY_NODE_FIELD(mergeActionList);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 83e67627ad..02ec025953 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -393,6 +393,7 @@ _outModifyTable(StringInfo str, const ModifyTable *node)
 	WRITE_NODE_FIELD(exclRelTlist);
 	WRITE_NODE_FIELD(mergeSourceTargetList);
 	WRITE_NODE_FIELD(mergeActionList);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 1667881127..426e825229 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1640,6 +1640,7 @@ _readModifyTable(void)
 	READ_NODE_FIELD(exclRelTlist);
 	READ_NODE_FIELD(mergeSourceTargetList);
 	READ_NODE_FIELD(mergeActionList);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 093ceaa867..cd75c59496 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -295,7 +295,8 @@ static ModifyTable *make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam);
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos);
 static GatherMerge *create_gather_merge_plan(PlannerInfo *root,
 						 GatherMergePath *best_path);
 
@@ -1108,7 +1109,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		 */
 		if (prunequal != NIL)
 			partpruneinfos = make_partition_pruneinfo(root,
-													  best_path->partitioned_rels,
+													  best_path->partitioned_rels, NIL,
 													  best_path->subpaths, prunequal);
 	}
 
@@ -1258,7 +1259,7 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		 */
 		if (prunequal != NIL)
 			partpruneinfos = make_partition_pruneinfo(root,
-													  best_path->partitioned_rels,
+													  best_path->partitioned_rels, NIL,
 													  best_path->subpaths, prunequal);
 	}
 
@@ -2484,6 +2485,7 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 	List	   *subplans = NIL;
 	ListCell   *subpaths,
 			   *subroots;
+	List	   *partpruneinfos = NIL;
 
 	/* Build the plan for each input path */
 	forboth(subpaths, best_path->subpaths,
@@ -2512,6 +2514,27 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (best_path->partitioned_rels != NIL)
+	{
+		int			partrelid = linitial_int(best_path->partitioned_rels);
+		RelOptInfo *rel = root->simple_rel_array[partrelid];
+		List	   *prunequal = NIL;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->resultRelations,
+													  best_path->subpaths, prunequal);
+	}
+
 	plan = make_modifytable(root,
 							best_path->operation,
 							best_path->canSetTag,
@@ -2527,7 +2550,8 @@ create_modifytable_plan(PlannerInfo *root, ModifyTablePath *best_path)
 							best_path->onconflict,
 							best_path->mergeSourceTargetList,
 							best_path->mergeActionList,
-							best_path->epqParam);
+							best_path->epqParam,
+							partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, &best_path->path);
 
@@ -6600,7 +6624,8 @@ make_modifytable(PlannerInfo *root,
 				 List *withCheckOptionLists, List *returningLists,
 				 List *rowMarks, OnConflictExpr *onconflict,
 				 List *mergeSourceTargetList,
-				 List *mergeActionList, int epqParam)
+				 List *mergeActionList, int epqParam,
+				 List *partpruneinfos)
 {
 	ModifyTable *node = makeNode(ModifyTable);
 	List	   *fdw_private_list;
@@ -6662,6 +6687,7 @@ make_modifytable(PlannerInfo *root,
 	node->mergeSourceTargetList = mergeSourceTargetList;
 	node->mergeActionList = mergeActionList;
 	node->epqParam = epqParam;
+	node->part_prune_infos = partpruneinfos;
 
 	/*
 	 * For each result relation that is a foreign table, allow the FDW to
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 421dc79cc4..5cfc665347 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1197,6 +1197,7 @@ inheritance_planner(PlannerInfo *root)
 	Query	   *parent_parse;
 	Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
 	PlannerInfo **parent_roots = NULL;
+	PlannerInfo *partition_root = NULL;
 
 	Assert(parse->commandType != CMD_INSERT);
 
@@ -1274,6 +1275,32 @@ inheritance_planner(PlannerInfo *root)
 		 * the ModifyTable node, if one is needed at all.
 		 */
 		partitioned_relids = bms_make_singleton(top_parentRTindex);
+
+		/*
+		 * For partitioned tables, since we're able to determine the minimum
+		 * set of partitions required much more easily than what we can do
+		 * with an inheritance hierarchy, we invoke the grouping_planner on
+		 * the entire given query in order to determine the minimum set of
+		 * partitions which will be required below.  This may mean that we
+		 * invoke the grouping planner far fewer times, as otherwise we'd have
+		 * to invoke it once for each partition.
+		 */
+
+		/*
+		 * Since the planner tends to scribble on the parse, we must make a
+		 * copy of it.  We also must make copies of the PlannerInfo and
+		 * PlannerGlobal since these will also be modified from the call to
+		 * grouping_planner.
+		 */
+		partition_root = makeNode(PlannerInfo);
+		partition_root->glob = makeNode(PlannerGlobal);
+
+		memcpy(partition_root, root, sizeof(PlannerInfo));
+		memcpy(partition_root->glob, root->glob, sizeof(PlannerGlobal));
+
+		partition_root->parse = copyObject(partition_root->parse);
+
+		grouping_planner(partition_root, true, 0.0 /* retrieve all tuples */ );
 	}
 
 	/*
@@ -1304,6 +1331,21 @@ inheritance_planner(PlannerInfo *root)
 		if (!bms_is_member(appinfo->parent_relid, parent_relids))
 			continue;
 
+		/*
+		 * If the target rel is a partitioned table then skip any child
+		 * partitions which were found to be dummies by the grouping_planner
+		 * call performed above.
+		 */
+		if (partition_root)
+		{
+			RelOptInfo *rel;
+
+			rel = find_base_rel(partition_root, appinfo->child_relid);
+
+			if (IS_DUMMY_REL(rel))
+				continue;
+		}
+
 		/*
 		 * expand_inherited_rtentry() always processes a parent before any of
 		 * that parent's children, so the parent_root for this relation should
@@ -1629,6 +1671,23 @@ inheritance_planner(PlannerInfo *root)
 		Assert(list_length(partitioned_rels) >= 1);
 	}
 
+	/*
+	 * The individual grouping_planner calls per partition above performed no
+	 * planning on the actual partitioned tables, however, in order to allow
+	 * partition pruning at run-time we must know the baserestrictinfo of each
+	 * partition.  We simply replace the RelOptInfos from the initial full
+	 * plan which was generated and replace the non-complete RelOptInfos which
+	 * are stored in root.
+	 */
+	if (partition_root)
+	{
+		int			i;
+
+		i = -1;
+		while ((i = bms_next_member(partitioned_relids, i)) >= 0)
+			root->simple_rel_array[i] = partition_root->simple_rel_array[i];
+	}
+
 	/* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
 	add_path(final_rel, (Path *)
 			 create_modifytable_path(root, final_rel,
diff --git a/src/backend/optimizer/util/partprune.c b/src/backend/optimizer/util/partprune.c
index 390efdc7a8..34e3684d4a 100644
--- a/src/backend/optimizer/util/partprune.c
+++ b/src/backend/optimizer/util/partprune.c
@@ -244,7 +244,10 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
  *
  * Here we generate partition pruning steps for 'prunequal' and also build a
  * data stucture which allows mapping of partition indexes into 'subpaths'
- * indexes.
+ * or 'resultRelations' indexes.
+ *
+ * If 'resultRelations' is non-NIL, then this List of relids is used to build
+ * the mapping structures.  Otherwise the 'subpaths' List is used.
  *
  * If no Params were found to match the partition key in any of the
  * 'partitioned_rels', then we return NIL.  In such a case run-time partition
@@ -252,7 +255,8 @@ generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
  */
 List *
 make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
-						 List *subpaths, List *prunequal)
+						 List *resultRelations, List *subpaths,
+						 List *prunequal)
 {
 	RelOptInfo *targetpart = NULL;
 	ListCell   *lc;
@@ -269,16 +273,34 @@ make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 	relid_subnode_map = palloc0(sizeof(int) * root->simple_rel_array_size);
 	relid_subpart_map = palloc0(sizeof(int) * root->simple_rel_array_size);
 
-	i = 1;
-	foreach(lc, subpaths)
+	/*
+	 * If 'resultRelations' are present then map these, otherwise we map the
+	 * 'subpaths' List.
+	 */
+	if (resultRelations != NIL)
+	{
+		i = 1;
+		foreach(lc, resultRelations)
+		{
+			int			resultrel = lfirst_int(lc);
+
+			Assert(resultrel < root->simple_rel_array_size);
+			relid_subnode_map[resultrel] = i++;
+		}
+	}
+	else
 	{
-		Path	   *path = (Path *) lfirst(lc);
-		RelOptInfo *pathrel = path->parent;
+		i = 1;
+		foreach(lc, subpaths)
+		{
+			Path	   *path = (Path *) lfirst(lc);
+			RelOptInfo *pathrel = path->parent;
 
-		Assert(IS_SIMPLE_REL(pathrel));
-		Assert(pathrel->relid < root->simple_rel_array_size);
+			Assert(IS_SIMPLE_REL(pathrel));
+			Assert(pathrel->relid < root->simple_rel_array_size);
 
-		relid_subnode_map[pathrel->relid] = i++;
+			relid_subnode_map[pathrel->relid] = i++;
+		}
 	}
 
 	/* Likewise for the partition_rels */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1986abaa9c..e37b5da0bf 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1027,6 +1027,8 @@ typedef struct EPQState
 } EPQState;
 
 
+struct PartitionPruning;
+
 /* ----------------
  *	 ResultState information
  * ----------------
@@ -1083,7 +1085,7 @@ typedef struct ModifyTableState
 	PlanState **mt_plans;		/* subplans (one per target rel) */
 	int			mt_nplans;		/* number of plans in the array */
 	int			mt_whichplan;	/* which one is being executed (0..n-1) */
-	ResultRelInfo *resultRelInfo;	/* per-subplan target relations */
+	ResultRelInfo **resultRelInfos; /* per-subplan target relations */
 	ResultRelInfo *rootResultRelInfo;	/* root target relation (partitioned
 										 * table root) */
 	List	  **mt_arowmarks;	/* per-subplan ExecAuxRowMark lists */
@@ -1109,6 +1111,14 @@ typedef struct ModifyTableState
 
 	/* Flags showing which subcommands are present INS/UPD/DEL/DO NOTHING */
 	int			mt_merge_subcommands;
+
+	/*
+	 * Details required to allow partitions to be eliminated from the scan, or
+	 * NULL if not possible.
+	 */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *mt_valid_subplans;	/* for runtime pruning, valid mt_plans
+									 * indexes to scan. */
 } ModifyTableState;
 
 /* ----------------
@@ -1130,7 +1140,6 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
-struct PartitionPruning;
 
 struct AppendState
 {
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index a7dbd31466..c1b6c21f70 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -242,6 +242,8 @@ typedef struct ModifyTable
 	List	   *exclRelTlist;	/* tlist of the EXCLUDED pseudo relation */
 	List	   *mergeSourceTargetList;
 	List	   *mergeActionList;	/* actions for MERGE */
+	List	   *part_prune_infos;	/* Mapping details for run-time subplan
+									 * pruning, one per partitioned_rels */
 } ModifyTable;
 
 /* ----------------
diff --git a/src/include/optimizer/partprune.h b/src/include/optimizer/partprune.h
index ff8d18847f..b7352d150c 100644
--- a/src/include/optimizer/partprune.h
+++ b/src/include/optimizer/partprune.h
@@ -21,6 +21,7 @@ extern List *generate_partition_pruning_steps(RelOptInfo *rel, List *clauses,
 								 bool *constfalse);
 
 extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
-						 List *subpaths, List *prunequal);
+						 List *resultRelations, List *subpaths,
+						 List *prunequal);
 
 #endif							/* PARTPRUNE_H */
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 792924fe0b..86d0c9cab6 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2795,4 +2795,114 @@ explain (analyze, costs off, summary off, timing off) select * from ma_test wher
 
 reset enable_seqscan;
 reset enable_sort;
+--
+-- Test run-time pruning of ModifyTable subnodes
+--
+-- Ensure only ma_test_p3 is scanned.
+explain (analyze, costs off, summary off, timing off) delete from ma_test where a = (select 29);
+                      QUERY PLAN                      
+------------------------------------------------------
+ Delete on ma_test (actual rows=0 loops=1)
+   Delete on ma_test_p1
+   Delete on ma_test_p2
+   Delete on ma_test_p3
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   ->  Seq Scan on ma_test_p1 (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on ma_test_p2 (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on ma_test_p3 (actual rows=1 loops=1)
+         Filter: (a = $0)
+         Rows Removed by Filter: 9
+(13 rows)
+
+-- Ensure no partitions are scanned.
+explain (analyze, costs off, summary off, timing off) delete from ma_test where a = (select 30);
+                  QUERY PLAN                   
+-----------------------------------------------
+ Delete on ma_test (actual rows=0 loops=1)
+   Delete on ma_test_p1
+   Delete on ma_test_p2
+   Delete on ma_test_p3
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   ->  Seq Scan on ma_test_p1 (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on ma_test_p2 (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on ma_test_p3 (never executed)
+         Filter: (a = $0)
+(12 rows)
+
+-- Ensure partition pruning works with an update of the partition key.
+explain (analyze, costs off, summary off, timing off) update ma_test set a = 29 where a = (select 1);
+                      QUERY PLAN                      
+------------------------------------------------------
+ Update on ma_test (actual rows=0 loops=1)
+   Update on ma_test_p1
+   Update on ma_test_p2
+   Update on ma_test_p3
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   ->  Seq Scan on ma_test_p1 (actual rows=1 loops=1)
+         Filter: (a = $0)
+         Rows Removed by Filter: 9
+   ->  Seq Scan on ma_test_p2 (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on ma_test_p3 (never executed)
+         Filter: (a = $0)
+(13 rows)
+
+-- Verify the above command
+select tableoid::regclass,a from ma_test where a = 29;
+  tableoid  | a  
+------------+----
+ ma_test_p3 | 29
+(1 row)
+
+truncate ma_test;
+prepare mt_q1 (int) as
+delete from ma_test where a > $1;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Delete on ma_test (actual rows=0 loops=1)
+   Delete on ma_test_p2
+   Delete on ma_test_p3
+   Subplans Pruned: 1
+   ->  Index Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=0 loops=1)
+         Index Cond: (a > $1)
+   ->  Index Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=0 loops=1)
+         Index Cond: (a > $1)
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Delete on ma_test (actual rows=0 loops=1)
+   Delete on ma_test_p3
+   Subplans Pruned: 2
+   ->  Index Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=0 loops=1)
+         Index Cond: (a > $1)
+(5 rows)
+
+-- Ensure ModifyTable behaves correctly when no subplans match exec params
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Delete on ma_test (actual rows=0 loops=1)
+   Delete on ma_test_p1
+   Subplans Pruned: 2
+   ->  Index Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a > $1)
+(5 rows)
+
 drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6f389bed1f..73dd6438a2 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -658,4 +658,38 @@ explain (analyze, costs off, summary off, timing off) select * from ma_test wher
 reset enable_seqscan;
 reset enable_sort;
 
+--
+-- Test run-time pruning of ModifyTable subnodes
+--
+
+-- Ensure only ma_test_p3 is scanned.
+explain (analyze, costs off, summary off, timing off) delete from ma_test where a = (select 29);
+
+-- Ensure no partitions are scanned.
+explain (analyze, costs off, summary off, timing off) delete from ma_test where a = (select 30);
+
+-- Ensure partition pruning works with an update of the partition key.
+explain (analyze, costs off, summary off, timing off) update ma_test set a = 29 where a = (select 1);
+
+-- Verify the above command
+select tableoid::regclass,a from ma_test where a = 29;
+
+truncate ma_test;
+
+prepare mt_q1 (int) as
+delete from ma_test where a > $1;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+-- Ensure ModifyTable behaves correctly when no subplans match exec params
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+
 drop table ma_test;
-- 
2.16.2.windows.1

#118Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#117)
4 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

I rebased this series on top of the committed version of the other patch.
Here's v22, with no other changes than rebasing. I did not include
0005, though.

Git changed the author tag for 0001. Not intentional.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

v22-0001-Provide-infrastructure-to-allow-partition-prunin.patchtext/plain; charset=us-asciiDownload
From e9b68957c39fc8b07a1df860216b30c32049cfc8 Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Fri, 6 Apr 2018 17:59:10 -0300
Subject: [PATCH v22 1/4] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/commands/explain.c       |  51 +++--
 src/backend/executor/execPartition.c | 419 +++++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c        |  20 ++
 src/backend/nodes/outfuncs.c         |  27 +++
 src/backend/nodes/readfuncs.c        |  19 ++
 src/backend/partitioning/partprune.c | 256 +++++++++++++++++++++
 src/include/executor/execPartition.h |  75 +++++++
 src/include/nodes/nodes.h            |   1 +
 src/include/nodes/primnodes.h        |  23 ++
 src/include/partitioning/partprune.h |  14 ++
 10 files changed, 887 insertions(+), 18 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 79f639d5e2..549622da93 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1811,28 +1811,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3173,18 +3178,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that was
+	 * specified in the plan means that some subnodes have been ignored per
+	 * instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index ad532773a3..c5330d3d00 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -39,6 +39,10 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_params_recurse(PartitionPruning *pprune,
+								 PartitionRelPruning *partrelprune,
+								 bool allparams,
+								 Bitmapset **validsubplans);
 
 
 /*
@@ -1272,3 +1276,418 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all.  This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruning:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruning details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruning
+ *		Setup the required data structure which is required for calling
+ *		ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These contexts can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruning *
+ExecSetupPartitionPruning(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionRelPruning *partrelprunes;
+	PartitionPruning *pprune;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	pprune = (PartitionPruning *) palloc(sizeof(PartitionPruning));
+	partrelprunes = (PartitionRelPruning *)
+		palloc(sizeof(PartitionRelPruning) *
+			   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	pprune->partrelpruning = partrelprunes;
+	pprune->npartrelpruning = list_length(partitionpruneinfo);
+	pprune->extparams = NULL;
+	pprune->execparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	pprune->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+												  "Partition Prune",
+												  ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionRelPruning *partrelprune = &partrelprunes[i];
+		PartitionPruneContext *context = &partrelprune->context;
+		PartitionDesc partdesc;
+		Relation	rel;
+		PartitionKey partkey;
+		int			partnatts;
+
+		partrelprune->present_parts = bms_copy(pinfo->present_parts);
+		partrelprune->subnode_map = palloc(sizeof(int) * pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(partrelprune->subnode_map, pinfo->subnode_map,
+			   sizeof(int) * pinfo->nparts);
+
+		/* We can use the subpart_map verbatim, since we never modify it */
+		partrelprune->subpart_map = pinfo->subpart_map;
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL; /* empty for now */
+
+		partrelprune->pruning_steps = pinfo->pruning_steps;
+
+		partrelprune->extparams = bms_copy(pinfo->extparams);
+		partrelprune->allparams = bms_union(pinfo->extparams,
+											pinfo->execparams);
+
+		/*
+		 * Accumulate the paramids which match the partitioned keys of all
+		 * partitioned tables.
+		 */
+		pprune->extparams = bms_add_members(pprune->extparams,
+											pinfo->extparams);
+
+		pprune->execparams = bms_add_members(pprune->execparams,
+											 pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the paramids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	pprune->allparams = bms_union(pprune->extparams, pprune->execparams);
+
+	return pprune;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'pprune'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so only once during their
+ * init plan.  The caller must only initialize the subnodes which are returned
+ * by this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'pprune' has any extparams.
+ *
+ * 'nsubnodes' must be passed as the total number of unpruned subnodes.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruning *pprune, int nsubnodes)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(pprune->extparams));
+
+	partrelprune = pprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
+
+	/* Determine which subnodes match the external params */
+	find_subplans_for_params_recurse(pprune, partrelprune, false, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(pprune->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been performed for external Params.
+	 */
+	bms_free(pprune->extparams);
+	pprune->extparams = NULL;
+
+	/*
+	 * If any subnodes were pruned, we must re-sequence the subnode indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subnodes which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubnodes)
+	{
+		int		   *new_subnode_indexes;
+		int			i;
+		int			newidx;
+
+		/*
+		 * First we must build an array which we can use to adjust the
+		 * existing subnode_map so that it contains the new subnode indexes.
+		 */
+		new_subnode_indexes = (int *) palloc(sizeof(int) * nsubnodes);
+		newidx = 0;
+		for (i = 0; i < nsubnodes; i++)
+		{
+			if (bms_is_member(i, result))
+				new_subnode_indexes[i] = newidx++;
+			else
+				new_subnode_indexes[i] = -1;	/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnode_map so
+		 * that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < pprune->npartrelpruning; i++)
+		{
+			PartitionRelPruning *partrelprune;
+			int			nparts;
+			int			j;
+
+			partrelprune = &pprune->partrelpruning[i];
+			nparts = partrelprune->context.nparts;
+
+			/*
+			 * We also need to reset the present_parts field so that it only
+			 * contains partition indexes that we actually still have subnodes
+			 * for.  It seems easier to build a fresh one, rather than trying
+			 * to update the existing one.
+			 */
+			bms_free(partrelprune->present_parts);
+			partrelprune->present_parts = NULL;
+
+			for (j = 0; j < nparts; j++)
+			{
+				int			oldidx = partrelprune->subnode_map[j];
+
+				/*
+				 * If this partition existed as a subnode then change the old
+				 * subnode index to the new subnode index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subnode list due to some subnodes being
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					partrelprune->subnode_map[j] = new_subnode_indexes[oldidx];
+
+					if (new_subnode_indexes[oldidx] >= 0)
+						partrelprune->present_parts =
+							bms_add_member(partrelprune->present_parts, j);
+				}
+			}
+		}
+
+		pfree(new_subnode_indexes);
+	}
+
+	return result;
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'pprune' for the current Param values.
+ *
+ * Here we utilize both external and exec Params for pruning.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruning *pprune)
+{
+	PartitionRelPruning *partrelprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	partrelprune = pprune->partrelpruning;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(pprune->prune_context);
+
+	find_subplans_for_params_recurse(pprune, partrelprune, true, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(pprune->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_params_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans and
+ *		ExecFindInitialMatchingSubPlans
+ */
+static void
+find_subplans_for_params_recurse(PartitionPruning *pprune,
+								 PartitionRelPruning *partrelprune,
+								 bool allparams,
+								 Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &partrelprune->context;
+	Bitmapset  *partset;
+	Bitmapset  *pruneparams;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * Use only external params unless we've been asked to also use exec
+	 * params too.
+	 */
+	if (allparams)
+		pruneparams = partrelprune->allparams;
+	else
+		pruneparams = partrelprune->extparams;
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(pruneparams))
+	{
+		context->safeparams = pruneparams;
+		partset = get_matching_partitions(context,
+										  partrelprune->pruning_steps);
+	}
+	else
+		partset = partrelprune->present_parts;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (partrelprune->subnode_map[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											partrelprune->subnode_map[i]);
+		else
+		{
+			int			partidx = partrelprune->subpart_map[i];
+
+			if (partidx != -1)
+				find_subplans_for_params_recurse(pprune,
+												 &pprune->partrelpruning[partidx],
+												 allparams, validsubplans);
+			else
+			{
+				/*
+				 * This could only happen if clauses used in planning where
+				 * more restrictive than those used here, or if the maps are
+				 * somehow corrupt.
+				 */
+				elog(ERROR, "partition missing from subplans");
+			}
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9287baaedc..38c5592d32 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2182,6 +2182,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(pruning_steps);
+	COPY_BITMAPSET_FIELD(present_parts);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnode_map, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpart_map, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5121,6 +5138,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 03a91c3352..f58efcd729 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1758,6 +1758,30 @@ _outMergeAction(StringInfo str, const MergeAction *node)
 	WRITE_NODE_FIELD(targetList);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(pruning_steps);
+	WRITE_BITMAPSET_FIELD(present_parts);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnode_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnode_map[i]);
+
+	appendStringInfoString(str, " :subpart_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpart_map[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3992,6 +4016,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2812dc9646..cf040adcfb 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1373,6 +1373,23 @@ _readMergeAction(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(pruning_steps);
+	READ_BITMAPSET_FIELD(present_parts);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnode_map, local_node->nparts);
+	READ_INT_ARRAY(subpart_map, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2645,6 +2662,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 959ee1643d..ff68dca1e9 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -21,10 +21,12 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -135,6 +137,7 @@ 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 bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static PruneStepResult *perform_pruning_base_step(PartitionPruneContext *context,
 						  PartitionPruneStepOp *opstep);
 static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *context,
@@ -145,6 +148,181 @@ static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
 						Expr *expr, Datum *value);
 
+/*
+ * make_partition_pruneinfo
+ *		Builds List of PartitionPruneInfos, one for each 'partitioned_rels'.
+ *		These can be used in the executor to allow additional partition
+ *		pruning to take place.
+ *
+ * Here we generate partition pruning steps for 'prunequal' and also build a
+ * data stucture which allows mapping of partition indexes into 'subpaths'
+ * indexes.
+ *
+ * If no Params were found to match the partition key in any of the
+ * 'partitioned_rels', then we return NIL.  In such a case run-time partition
+ * pruning would be useless.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal)
+{
+	RelOptInfo *targetpart = NULL;
+	ListCell   *lc;
+	List	   *pinfolist = NIL;
+	int		   *relid_subnode_map;
+	int		   *relid_subpart_map;
+	int			i;
+	bool		gotparam = false;
+
+	/*
+	 * Allocate two arrays to store the 1-based indexes of the 'subpaths' and
+	 * 'partitioned_rels' by relid.
+	 */
+	relid_subnode_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+	relid_subpart_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+
+	i = 1;
+	foreach(lc, subpaths)
+	{
+		Path	   *path = (Path *) lfirst(lc);
+		RelOptInfo *pathrel = path->parent;
+
+		Assert(IS_SIMPLE_REL(pathrel));
+		Assert(pathrel->relid < root->simple_rel_array_size);
+
+		relid_subnode_map[pathrel->relid] = i++;
+	}
+
+	/* Likewise for the partition_rels */
+	i = 1;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		relid_subpart_map[rti] = i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		RangeTblEntry *rte;
+		Bitmapset  *present_parts;
+		int			nparts = subpart->nparts;
+		int		   *subnode_map;
+		int		   *subpart_map;
+		List	   *partprunequal;
+		List	   *pruning_steps;
+		bool		contradictory;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (!targetpart)
+		{
+			targetpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+				adjust_appendrel_attrs_multilevel(root,
+												  (Node *) prunequal,
+												  subpart->relids,
+												  targetpart->relids);
+		}
+
+		pruning_steps = gen_partprune_steps(subpart, partprunequal,
+											&contradictory);
+
+		if (contradictory)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered as the Param
+			 * values are unknown during planning.  Anyway, we'd better do
+			 * something sane here, so let's just disable run-time pruning.
+			 */
+			return NIL;
+		}
+
+		subnode_map = (int *) palloc(nparts * sizeof(int));
+		subpart_map = (int *) palloc(nparts * sizeof(int));
+		present_parts = NULL;
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present in
+		 * the subpaths List will be set to -1, and any sub-partitioned table
+		 * which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = relid_subnode_map[partrel->relid] - 1;
+			int			subpartidx = relid_subpart_map[partrel->relid] - 1;
+
+			subnode_map[i] = subnodeidx;
+			subpart_map[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				present_parts = bms_add_member(present_parts, i);
+		}
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+		pinfo->pruning_steps = pruning_steps;
+		pinfo->present_parts = present_parts;
+		pinfo->nparts = nparts;
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+		pinfo->subnode_map = subnode_map;
+		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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pruning_steps);
+
+		pinfolist = lappend(pinfolist, pinfo);
+	}
+
+	pfree(relid_subnode_map);
+	pfree(relid_subpart_map);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
 
 /*
  * gen_partprune_steps
@@ -240,6 +418,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	context.nparts = rel->nparts;
 	context.boundinfo = rel->boundinfo;
 
+	/* Not valid when being called from the planner */
+	context.planstate = NULL;
+	context.safeparams = NULL;
+
 	/* Actual pruning happens here. */
 	partindexes = get_matching_partitions(&context, pruning_steps);
 
@@ -2474,6 +2656,57 @@ get_matching_range_bounds(PartitionPruneContext *context,
 }
 
 /*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell   *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							 (int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
+/*
  * perform_pruning_base_step
  *		Determines the indexes of datums that satisfy conditions specified in
  *		'opstep'.
@@ -2774,6 +3007,29 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+
+			/*
+			 * 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;
+				bool		isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+												   context->planstate->ps_ExprContext,
+												   &isNull);
+				if (isNull)
+					return false;
+
+				return true;
+			}
+
 		default:
 			break;
 	}
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 9f55f6409e..7ee4bc7365 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "partitioning/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,75 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionRelPruning - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows the planner's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'pruning_steps' will be found within.
+ *
+ * subnode_map					An array containing the subnode index which
+ *								matches this partition index, or -1 if the
+ *								subnode has been pruned already.
+ * subpart_map					An array containing the offset into the
+ *								'partrelpruning' array in PartitionPruning, or
+ *								-1 if there is no such element in that array.
+ * present_parts				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * pruning_steps				Contains a list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ * extparams					Contains paramids of external params found
+ *								matching partition keys in 'pruning_steps'.
+ * allparams					As 'extparams' but also including exec params.
+ *-----------------------
+ */
+typedef struct PartitionRelPruning
+{
+	int		   *subnode_map;
+	int		   *subpart_map;
+	Bitmapset  *present_parts;
+	PartitionPruneContext context;
+	List	   *pruning_steps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionRelPruning;
+
+/*-----------------------
+ * PartitionPruning - Encapsulates a hierarchy of PartitionRelPruning
+ * structs and also stores all paramids which were found to match the
+ * partition keys of each partition.  This struct can be attached to node
+ * types which support arbitrary Lists of subnodes containing partitions to
+ * allow subnodes to be eliminated due to the clauses being unable to match
+ * to any tuple that the subnode could possibly produce.
+ *
+ * partrelpruning		Array of PartitionRelPruning for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * npartrelpruning		Number of items in 'partrelpruning' array.
+ * 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
+ *						partition key in each of the contained
+ *						PartitionRelPruning structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of 'extparams' and 'execparams', saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruning
+{
+	PartitionRelPruning *partrelpruning;
+	int			npartrelpruning;
+	MemoryContext prune_context;
+	Bitmapset  *extparams;
+	Bitmapset  *execparams;
+	Bitmapset  *allparams;
+} PartitionPruning;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -127,5 +197,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot *new_slot,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(PartitionTupleRouting *proute);
+extern PartitionPruning *ExecSetupPartitionPruning(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruning *partprune);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruning *partprune,
+								int nsubnodes);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 4fc2de7184..defdbae507 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -196,6 +196,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index ff5c4ff8e4..f90aa7b2a1 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1581,4 +1581,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;			/* Oid of partition rel */
+	List	   *pruning_steps;	/* List of PartitionPruneStep */
+	Bitmapset  *present_parts;	/* Indexes of all partitions which subnodes
+								 * are present for. */
+	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 */
+	Bitmapset  *extparams;		/* All external paramids seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec paramids seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index 52fadc7caf..3e8cc76c3d 100644
--- a/src/include/partitioning/partprune.h
+++ b/src/include/partitioning/partprune.h
@@ -37,9 +37,23 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow params
+	 * found matching the partition key to be evaulated.
+	 */
+	PlanState  *planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until after init plan.
+	 */
+	Bitmapset  *safeparams;
 } PartitionPruneContext;
 
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal);
 extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern Bitmapset *get_matching_partitions(PartitionPruneContext *context,
 						List *pruning_steps);
-- 
2.11.0

v22-0002-Add-bms_prev_member-function.patchtext/plain; charset=us-asciiDownload
From 34b6ad594e80196674506d21809126b230c11a0d Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:17:58 +1200
Subject: [PATCH v22 2/4] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1089,6 +1111,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 }
 
 /*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
+/*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
  * Note: we must ensure that any two bitmapsets that are bms_equal() will
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.11.0

v22-0003-Allow-unneeded-Append-subnodes-to-be-pruned-at-e.patchtext/plain; charset=us-asciiDownload
From 2e39a92d02fbf7bd39b9750c42fe1f1d64ca78d0 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:23:16 +1200
Subject: [PATCH v22 3/4] Allow unneeded Append subnodes to be pruned at
 execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  262 ++++--
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   15 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1803 insertions(+), 89 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..6e21b4b708 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -82,6 +83,7 @@ static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +101,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -113,54 +117,117 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
 	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
-	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+	appendstate->as_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *pprune;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		pprune = ExecSetupPartitionPruning(&appendstate->ps,
+										   node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(pprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune,
+															list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan, but we set a noop flag so that we never actually
+			 * bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(pprune->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->partition_pruning = pprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		appendstate->partition_pruning = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+											 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the appendplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
-	/*
-	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
-	 */
-	appendstate->ps.ps_ProjInfo = NULL;
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
 
 	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
+	 * Miscellaneous initialization
 	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+
+	appendstate->ps.ps_ProjInfo = NULL;
+
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -251,6 +318,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +350,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +440,40 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
+
+	/* Handle the case for when all subplans were pruned */
+	if (node->as_noopscan)
+		return false;
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
+	{
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+		whichplan = -1;
+	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
 
 	if (ScanDirectionIsForward(node->ps.state->es_direction))
-	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
-	}
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -411,10 +502,33 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
 	else
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -467,6 +581,25 @@ choose_next_subplan_for_worker(AppendState *node)
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	else if (node->as_noopscan)
+	{
+		/* Handle the case for when all subplans were pruned */
+		LWLockRelease(&pstate->pa_lock);
+		return false;
+	}
+
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+			ExecFindMatchingSubPlans(node->partition_pruning);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +665,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int			i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->partition_pruning);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 38c5592d32..765f290bc7 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -248,6 +248,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index 51c418778a..4c309d236a 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3806,32 +3806,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3881,15 +3881,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index f58efcd729..c274adaacc 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -419,6 +419,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index cf040adcfb..93785e3bdf 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1692,6 +1692,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 65a34a255d..3ba3f87eb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1604,7 +1604,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1646,8 +1646,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1695,7 +1695,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1758,7 +1758,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2024,7 +2024,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99d0736029..7f72bd0726 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
+#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1041,6 +1042,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1078,6 +1081,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1086,7 +1120,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5382,7 +5417,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5394,7 +5430,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 008492bad5..421dc79cc4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3920,7 +3920,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6852,8 +6853,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 67e47887fc..2ce4d4496d 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 416b3f9578..bd9442c22d 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3574,7 +3592,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ff63d179b2..b63c0c5329 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1114,8 +1114,15 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1)
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
+ *		noopscan			true if partition pruning proved that none of the
+ *							appendplans can contain a record to satisfy this
+ *							query.
  * ----------------
  */
 
@@ -1123,6 +1130,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruning;
 
 struct AppendState
 {
@@ -1132,7 +1140,10 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
+	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0a797f0a05..c3e5c2c79f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -256,6 +256,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 895bf6959d..4f65686d9b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 2d77b3edd4..0a2517125c 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1516,3 +1516,1138 @@ explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 (9 rows)
 
 drop table hp;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index ad5177715c..eefbf32e93 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -274,3 +274,347 @@ explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and
 explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
 
 drop table hp;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.11.0

v22-0004-Allow-unneeded-MergeAppend-s-subnodes-to-be-prun.patchtext/plain; charset=us-asciiDownload
From 6943aa77e5113d07c670b56c0a5cd5d26f30e286 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Fri, 6 Apr 2018 22:29:29 +1200
Subject: [PATCH v22 4/4] Allow unneeded MergeAppend's subnodes to be pruned at
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   5 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 353 insertions(+), 21 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..31bc85b4b1 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,10 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -79,18 +82,88 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
 	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
-	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruning *pprune;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		pprune = ExecSetupPartitionPruning(&mergestate->ps,
+											  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(pprune->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(pprune,
+															list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(pprune->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->partition_pruning = pprune;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		mergestate->partition_pruning = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +174,24 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +249,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+				ExecFindMatchingSubPlans(node->partition_pruning);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +376,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->partition_pruning &&
+		bms_overlap(node->ps.chgParam,
+					node->partition_pruning->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 765f290bc7..0c2041bd2e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -276,6 +276,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index c274adaacc..14567048be 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -451,6 +451,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 93785e3bdf..2e9da37fc3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1714,6 +1714,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 7f72bd0726..093ceaa867 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1144,6 +1144,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1229,8 +1231,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index b63c0c5329..1986abaa9c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1155,6 +1155,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		partition_pruning	details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		ms_valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *							scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1167,6 +1173,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruning *partition_pruning;
+	Bitmapset  *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c3e5c2c79f..a7dbd31466 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -280,6 +280,11 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 0a2517125c..792924fe0b 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2651,3 +2651,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index eefbf32e93..6f389bed1f 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -618,3 +618,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.11.0

#119Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#118)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Alvaro Herrera wrote:

I rebased this series on top of the committed version of the other patch.
Here's v22, with no other changes than rebasing. I did not include
0005, though.

Apologies, I forgot to "git add" one fixup for 0001.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

fixup.patchtext/plain; charset=us-asciiDownload
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 093ceaa867..aac05917ee 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,7 +29,6 @@
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
 #include "optimizer/cost.h"
-#include "optimizer/partprune.h"
 #include "optimizer/paths.h"
 #include "optimizer/placeholder.h"
 #include "optimizer/plancat.h"
@@ -42,6 +41,7 @@
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
+#include "partitioning/partprune.h"
 #include "utils/lsyscache.h"
 
 
#120David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#119)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 09:29, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Alvaro Herrera wrote:

I rebased this series on top of the committed version of the other patch.
Here's v22, with no other changes than rebasing. I did not include
0005, though.

Apologies, I forgot to "git add" one fixup for 0001.

0003 I think.

I'm looking over the rebased patches now.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#121David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#120)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 10:45, David Rowley <david.rowley@2ndquadrant.com> wrote:

I'm looking over the rebased patches now.

I've made a complete read of 0001 and 0002 so far.

Your rebase looks fine.

After the complete read, I only have the following comments:

0001:

1. missing "the" before "partition key":

* Extract Params matching partition key and record if we got any.

2. Is this the property name we're going to stick with:

ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);

Other ideas are: "Subplans Removed"

3. In the following comment I've used the word "hierarchy", but maybe
we need to add the word "flattened" before it.

* PartitionPruning - Encapsulates a hierarchy of PartitionRelPruning

4. Comment mentions "after init plan", but really we can only know the
value of an exec param during actual execution. So:

* Parameters that are safe to be used for partition pruning. execparams
* are not safe to use until after init plan.

maybe better as:

* Parameters that are safe to be used for partition pruning. execparams
* are not safe to use until the executor is running.

0002:

Looks fine. But if I was committing this, to give me confidence, I'd
want to know how the left_most_one table was generated.

I used:

#include <stdio.h>

int main(void)
{
int i = 1;

printf("0, ");
while (i < 256)
{
printf("%d, ", 31 - __builtin_clz(i));
if ((i & 0xf) == 0xf)
putchar('\n');
i++;
}
return 0;
}

Continuing to read 0003 and 0004 now.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#122David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#121)
2 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 12:03, David Rowley <david.rowley@2ndquadrant.com> wrote:

Continuing to read 0003 and 0004 now.

0003:

1. "setup" -> "set"

/* If run-time partition pruning is enabled, then setup that up now */

2. We should be able to get rid of as_noopscan and just have another
special negative value for as_whichplan.

I've attached a patch to do this.

3. I've forgotten to drop table boolvalues in the tests.

Patched attached to fix.

0004:

1. "ms_valid_subplans" -> "valid_subplans" in:

* ms_valid_subplans for runtime pruning, valid mergeplans indexes to
* scan.

All the other fields are not being prefixed with ms_ in these comments.

Everything else looks fine from my point of view.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

nodeAppend_get_rid_of_as_noopscan.patchapplication/octet-stream; name=nodeAppend_get_rid_of_as_noopscan.patchDownload
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6e21b4b708..03933cf694 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -78,6 +78,7 @@ struct ParallelAppendState
 };
 
 #define INVALID_SUBPLAN_INDEX		-1
+#define NO_MATCHING_SUBPLANS		-2
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
@@ -122,7 +123,9 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->as_noopscan = false;
+
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If run-time partition pruning is enabled, then setup that up now */
 	if (node->part_prune_infos != NIL)
@@ -150,12 +153,12 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 			 * explain.c requires an Append to have at least one subplan in
 			 * order for it to properly determine the Vars in that subplan's
 			 * targetlist.  We sidestep this issue by just initializing the
-			 * first subplan, but we set a noop flag so that we never actually
-			 * bother scanning it.
+			 * first subplan, but set as_whichplan to a special value which
+			 * indicates that no plans match.
 			 */
 			if (bms_is_empty(validsubplans))
 			{
-				appendstate->as_noopscan = true;
+				appendstate->as_whichplan = NO_MATCHING_SUBPLANS;
 
 				/* Mark the first as valid so that it's initialized below */
 				validsubplans = bms_make_singleton(0);
@@ -226,9 +229,6 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 
 	appendstate->ps.ps_ProjInfo = NULL;
 
-	/* Let choose_next_subplan_* function handle setting the first subplan */
-	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
-
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
 
@@ -245,11 +245,22 @@ static TupleTableSlot *
 ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
+	int			whichplan = node->as_whichplan;
 
-	/* If no subplan has been chosen, we must choose one before proceeding. */
-	if (node->as_whichplan == INVALID_SUBPLAN_INDEX &&
-		!node->choose_next_subplan(node))
-		return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+	if (whichplan < 0)
+	{
+		/*
+		 * If no subplan has been chosen, we must choose one before
+		 * proceeding.
+		 */
+		if (whichplan == INVALID_SUBPLAN_INDEX &&
+			!node->choose_next_subplan(node))
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Nothing to do if there are no matching subplans */
+		else if (whichplan == NO_MATCHING_SUBPLANS)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+	}
 
 	for (;;)
 	{
@@ -442,9 +453,8 @@ choose_next_subplan_locally(AppendState *node)
 	int			whichplan = node->as_whichplan;
 	int			nextplan;
 
-	/* Handle the case for when all subplans were pruned */
-	if (node->as_noopscan)
-		return false;
+	/* We should never be called when there are no subplans */
+	Assert(whichplan != NO_MATCHING_SUBPLANS);
 
 	/*
 	 * If first call then have the bms member function choose the first valid
@@ -495,6 +505,9 @@ choose_next_subplan_for_leader(AppendState *node)
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
 
+	/* We should never be called when there are no subplans */
+	Assert(node->as_whichplan != NO_MATCHING_SUBPLANS);
+
 	LWLockAcquire(&pstate->pa_lock, LW_EXCLUSIVE);
 
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
@@ -502,12 +515,6 @@ choose_next_subplan_for_leader(AppendState *node)
 		/* Mark just-completed subplan as finished. */
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 	}
-	else if (node->as_noopscan)
-	{
-		/* Handle the case for when all subplans were pruned */
-		LWLockRelease(&pstate->pa_lock);
-		return false;
-	}
 	else
 	{
 		/* Start with last subplan. */
@@ -575,19 +582,15 @@ choose_next_subplan_for_worker(AppendState *node)
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
 
+	/* We should never be called when there are no subplans */
+	Assert(node->as_whichplan != NO_MATCHING_SUBPLANS);
+
 	LWLockAcquire(&pstate->pa_lock, LW_EXCLUSIVE);
 
 	/* Mark just-completed subplan as finished. */
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
-	else if (node->as_noopscan)
-	{
-		/* Handle the case for when all subplans were pruned */
-		LWLockRelease(&pstate->pa_lock);
-		return false;
-	}
-
 	/*
 	 * If we've yet to determine the valid subplans for these parameters then
 	 * do so now.  If run-time pruning is disabled then the valid subplans
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 1986abaa9c..fccc38cbe7 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1115,14 +1115,12 @@ typedef struct ModifyTableState
  *	 AppendState information
  *
  *		nplans				how many plans are in the array
- *		whichplan			which plan is being executed (0 .. n-1)
+ *		whichplan			which plan is being executed (0 .. n-1), or a
+ *							special negative value. See nodeAppend.c.
  *		partition_pruning	details required to allow partitions to be
  *							eliminated from the scan, or NULL if not possible.
  *		valid_subplans		for runtime pruning, valid appendplans indexes to
  *							scan.
- *		noopscan			true if partition pruning proved that none of the
- *							appendplans can contain a record to satisfy this
- *							query.
  * ----------------
  */
 
@@ -1143,7 +1141,6 @@ struct AppendState
 	struct PartitionPruning *partition_pruning;
 	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
-	bool		as_noopscan;	/* true if no subplans need scanned */
 };
 
 /* ----------------
drop_table_boolvalues.patchapplication/octet-stream; name=drop_table_boolvalues.patchDownload
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 792924fe0b..206d83dd96 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2650,7 +2650,7 @@ select * from boolp where a = (select value from boolvalues where not value);
          Filter: (a = $0)
 (9 rows)
 
-drop table boolp;
+drop table boolp, boolvalues;
 --
 -- Test run-time pruning of MergeAppend subnodes
 --
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 6f389bed1f..2f2fdf6e57 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -617,7 +617,7 @@ select * from boolp where a = (select value from boolvalues where value);
 explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
-drop table boolp;
+drop table boolp, boolvalues;
 
 --
 -- Test run-time pruning of MergeAppend subnodes
#123Amit Langote
amitlangote09@gmail.com
In reply to: David Rowley (#122)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Apr 7, 2018 at 11:26 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Everything else looks fine from my point of view.

Me too, although I still think having struct names PartitionPruning
and PartitionRelPruning is going to be a bit confusing. We should
think about naming the latter to something else. I suggested
PartitionPruningDispatch(Data), but David doesn't seem to like it.
Maybe, PartitionPruneState, because it parallels the
PartitionPruneInfo that comes from the planner for every partitioned
table in the tree.

Thanks,
Amit

#124Andres Freund
andres@anarazel.de
In reply to: Amit Langote (#123)
Re: [HACKERS] Runtime Partition Pruning

On 2018-04-07 13:26:51 +0900, Amit Langote wrote:

On Sat, Apr 7, 2018 at 11:26 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Everything else looks fine from my point of view.

Me too, although I still think having struct names PartitionPruning
and PartitionRelPruning is going to be a bit confusing. We should
think about naming the latter to something else. I suggested
PartitionPruningDispatch(Data), but David doesn't seem to like it.
Maybe, PartitionPruneState, because it parallels the
PartitionPruneInfo that comes from the planner for every partitioned
table in the tree.

I've not followed this thread/feature at all, but I don't find the
comments atop partprune.c even remotely sufficient. Unless there's an
README hidden or such hidden somewhere?

Greetings,

Andres Freund

#125David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#123)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 16:26, Amit Langote <amitlangote09@gmail.com> wrote:

Maybe, PartitionPruneState, because it parallels the
PartitionPruneInfo that comes from the planner for every partitioned
table in the tree.

I like that.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#126David Rowley
david.rowley@2ndquadrant.com
In reply to: Andres Freund (#124)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 16:31, Andres Freund <andres@anarazel.de> wrote:

I've not followed this thread/feature at all, but I don't find the
comments atop partprune.c even remotely sufficient. Unless there's an
README hidden or such hidden somewhere?

There's not a README file. The comments for partprune.c, do you want
this to explain more about how partition pruning works or how this
patch uses the existing code?

Probably if we need to explain more there about how pruning works then
it should be a fixup patch to 9fdb675fc, no?

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#127Amit Langote
amitlangote09@gmail.com
In reply to: Andres Freund (#124)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Apr 7, 2018 at 1:31 PM, Andres Freund <andres@anarazel.de> wrote:

On 2018-04-07 13:26:51 +0900, Amit Langote wrote:

On Sat, Apr 7, 2018 at 11:26 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Everything else looks fine from my point of view.

Me too, although I still think having struct names PartitionPruning
and PartitionRelPruning is going to be a bit confusing. We should
think about naming the latter to something else. I suggested
PartitionPruningDispatch(Data), but David doesn't seem to like it.
Maybe, PartitionPruneState, because it parallels the
PartitionPruneInfo that comes from the planner for every partitioned
table in the tree.

I've not followed this thread/feature at all, but I don't find the
comments atop partprune.c even remotely sufficient. Unless there's an
README hidden or such hidden somewhere?

Sorry there isn't a README and I agree partprune.c's header comment
could be improved quite a bit.

Just to be clear, that's the fault of the patch that was already
committed earlier today (9fdb675fc "Faster partition pruning"), not
this patch, which just extends partition.c's functionality to
implement additional planner and executor support for runtime pruning.

I'm drafting a patch that expands the partprune.c comment and will post shortly.

Thanks,
Amit

#128Amit Langote
amitlangote09@gmail.com
In reply to: David Rowley (#126)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Apr 7, 2018 at 1:58 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Probably if we need to explain more there about how pruning works then
it should be a fixup patch to 9fdb675fc, no?

Yes, I just replied and working on a patch.

Thanks,
Amit

#129Andres Freund
andres@anarazel.de
In reply to: David Rowley (#126)
Re: [HACKERS] Runtime Partition Pruning

On 2018-04-07 16:58:01 +1200, David Rowley wrote:

On 7 April 2018 at 16:31, Andres Freund <andres@anarazel.de> wrote:

I've not followed this thread/feature at all, but I don't find the
comments atop partprune.c even remotely sufficient. Unless there's an
README hidden or such hidden somewhere?

There's not a README file. The comments for partprune.c, do you want
this to explain more about how partition pruning works or how this
patch uses the existing code?

Primarily the first. This isn't trivial straightforward code.

Probably if we need to explain more there about how pruning works then
it should be a fixup patch to 9fdb675fc, no?

Yea, it's about that. Sorry for accidentally jumping on the wrong
thread.

Greetings,

Andres Freund

#130Amit Langote
amitlangote09@gmail.com
In reply to: Amit Langote (#127)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Apr 7, 2018 at 1:58 PM, Amit Langote <amitlangote09@gmail.com> wrote:

On Sat, Apr 7, 2018 at 1:31 PM, Andres Freund <andres@anarazel.de> wrote:

I've not followed this thread/feature at all, but I don't find the
comments atop partprune.c even remotely sufficient. Unless there's an
README hidden or such hidden somewhere?

Sorry there isn't a README and I agree partprune.c's header comment
could be improved quite a bit.

Just to be clear, that's the fault of the patch that was already
committed earlier today (9fdb675fc "Faster partition pruning"), not
this patch, which just extends partition.c's functionality to
implement additional planner and executor support for runtime pruning.

I'm drafting a patch that expands the partprune.c comment and will post shortly.

See if the attached makes it any better.

Now I know we don't have the runtime pruning in yet, but since the
proposed patch would extend its functionality I have included its
description in the comment.

Thanks,
Amit

Attachments:

expand-partprune-header-comment.patchapplication/octet-stream; name=expand-partprune-header-comment.patchDownload
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index ff68dca..9998583 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -1,10 +1,36 @@
 /*-------------------------------------------------------------------------
  *
  * partprune.c
- *		Parses clauses attempting to match them up to partition keys of a
- *		given relation and generates a set of "pruning steps", which can be
- *		later "executed" either from the planner or the executor to determine
- *		the minimum set of partitions which match the given clauses.
+ *		This module implements partition pruning using the information
+ *		contained in table's partition descriptor.
+ *
+ * Functionality of this module can be invoked either from the planner or the
+ * executor.  During planning, clauses that can be matched to the table's
+ * partition key are turned into a set of "pruning steps", which are then
+ * executed to produce a set of indexes of partitions whose bounds satisfy the
+ * set of clauses from which a given step was constructed.  When invoked from
+ * the executor, "pruning steps" that were generated during planning are
+ * executed using expressions whose values could only be known during
+ * execution, such as Params.
+ *
+ * There are two kinds of pruning steps -- a "base" pruning step, which
+ * contains information extracted from one of more OpExpr clauses that are
+ * matched to the (possibly multi-column) partition key, such as the
+ * expressions whose values to match against partition bounds and operator
+ * strategy to assume when determining the set of partition bounds that would
+ * match those expressions.  Whereas, a "combine" pruning step is constructed
+ * for BoolExpr clauses which simply combines the outputs of the steps
+ * corresponding to its argument clauses using the appropriate combination
+ * method.  All steps that are constructed are executed in succession such
+ * that for any "combine" step, all of the steps whose output it depends on
+ * are executed first and their ouput preserved.  Refer to
+ * gen_partprune_steps_internal() for more details.
+ *
+ * A base pruning step may consist of Param expressions whose values are
+ * only made available during execution, in which case, pruning cannot occur
+ * during planning itself or it occurs only partially with other expressions
+ * whose values are known during planning.  Such steps are included with the
+ * plan, so the they can be executed at an appropriate time during execution.
  *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
#131David Rowley
david.rowley@2ndquadrant.com
In reply to: Amit Langote (#123)
4 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 7 April 2018 at 16:26, Amit Langote <amitlangote09@gmail.com> wrote:

On Sat, Apr 7, 2018 at 11:26 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

Everything else looks fine from my point of view.

Me too, although I still think having struct names PartitionPruning
and PartitionRelPruning is going to be a bit confusing. We should
think about naming the latter to something else. I suggested
PartitionPruningDispatch(Data), but David doesn't seem to like it.
Maybe, PartitionPruneState, because it parallels the
PartitionPruneInfo that comes from the planner for every partitioned
table in the tree.

Ok, so I've gone and done this.

PartitionPruning has become PartitionPruneState
PartitionRelPruning has become PartitionPruningData

I've changed pointers to PartitionPruneStates to be named prunestate,
sometimes having the node prefix; as_, ma_, in these cases prune and
state are separated with a _ which seems to be the general rule for
executor state struct members.

Generally, pointers to PartitionPruningData are now named pprune.
Hopefully, that's ok, as this was the name previously used for
PartitionPruning pointers.

I applied the patch to get rid of as_noop_scan in favour of using a
special as_whichplan value. There was already one special value
(INVALID_SUBPLAN_INDEX), so seemed better to build on that rather than
inventing something new. This also means we don't have to make the
AppendState struct and wider too, which seems like a good thing to try
to do.

I made all the fixups which I mentioned in my review earlier and also
re-removed the resultRelation parameter from make_partition_pruneinfo.
It sneaked back into v22.

v23 is attached.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

v23-0001-Provide-infrastructure-to-allow-partition-prunin.patchapplication/octet-stream; name=v23-0001-Provide-infrastructure-to-allow-partition-prunin.patchDownload
From 4edc801bba83985e8ee0b2718feb7a277425e028 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sat, 7 Apr 2018 20:23:59 +1200
Subject: [PATCH v23 1/4] Provide infrastructure to allow partition pruning
 during execution

The query planner supports eliminating partitions of a partitioned table
during query planning.  This has its limitations as it can only perform the
elimination using clauses which can be evaluated during planning.  Allowing
this partition elimination to occur during execution allows the values of
Params to be used for elimination too, thus opening the door for PREPAREd
statements to have unneeded partitions pruned too.

The infrastructure provided here permits the building of a data structure
which is able to perform the translation of the matching partition IDs as is
returned by the existing partition pruning code into the List index of a
subpaths list, as exist in node types such as Append, MergeAppend and
ModifyTable.  This allows us to translate a list of clauses into a Bitmapset
of all the subpath indexes which must be included to satisfy the clause list.

This commit does not add support for any node types. Support for this will
arrive in follow-up commits.
---
 src/backend/commands/explain.c       |  51 +++--
 src/backend/executor/execPartition.c | 418 +++++++++++++++++++++++++++++++++++
 src/backend/nodes/copyfuncs.c        |  20 ++
 src/backend/nodes/outfuncs.c         |  27 +++
 src/backend/nodes/readfuncs.c        |  19 ++
 src/backend/partitioning/partprune.c | 256 +++++++++++++++++++++
 src/include/executor/execPartition.h |  77 +++++++
 src/include/nodes/nodes.h            |   1 +
 src/include/nodes/primnodes.h        |  23 ++
 src/include/partitioning/partprune.h |  14 ++
 10 files changed, 888 insertions(+), 18 deletions(-)

diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 79f639d5e2..549622da93 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -118,8 +118,8 @@ static void ExplainModifyTarget(ModifyTable *plan, ExplainState *es);
 static void ExplainTargetRel(Plan *plan, Index rti, ExplainState *es);
 static void show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
 					  ExplainState *es);
-static void ExplainMemberNodes(List *plans, PlanState **planstates,
-				   List *ancestors, ExplainState *es);
+static void ExplainMemberNodes(PlanState **planstates, int nsubnodes,
+				   int nplans, List *ancestors, ExplainState *es);
 static void ExplainSubPlans(List *plans, List *ancestors,
 				const char *relationship, ExplainState *es);
 static void ExplainCustomChildren(CustomScanState *css,
@@ -1811,28 +1811,33 @@ ExplainNode(PlanState *planstate, List *ancestors,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			ExplainMemberNodes(((ModifyTable *) plan)->plans,
-							   ((ModifyTableState *) planstate)->mt_plans,
+			ExplainMemberNodes(((ModifyTableState *) planstate)->mt_plans,
+							   ((ModifyTableState *) planstate)->mt_nplans,
+							   list_length(((ModifyTable *) plan)->plans),
 							   ancestors, es);
 			break;
 		case T_Append:
-			ExplainMemberNodes(((Append *) plan)->appendplans,
-							   ((AppendState *) planstate)->appendplans,
+			ExplainMemberNodes(((AppendState *) planstate)->appendplans,
+							   ((AppendState *) planstate)->as_nplans,
+							   list_length(((Append *) plan)->appendplans),
 							   ancestors, es);
 			break;
 		case T_MergeAppend:
-			ExplainMemberNodes(((MergeAppend *) plan)->mergeplans,
-							   ((MergeAppendState *) planstate)->mergeplans,
+			ExplainMemberNodes(((MergeAppendState *) planstate)->mergeplans,
+							   ((MergeAppendState *) planstate)->ms_nplans,
+							   list_length(((MergeAppend *) plan)->mergeplans),
 							   ancestors, es);
 			break;
 		case T_BitmapAnd:
-			ExplainMemberNodes(((BitmapAnd *) plan)->bitmapplans,
-							   ((BitmapAndState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapAndState *) planstate)->bitmapplans,
+							   ((BitmapAndState *) planstate)->nplans,
+							   list_length(((BitmapAnd *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_BitmapOr:
-			ExplainMemberNodes(((BitmapOr *) plan)->bitmapplans,
-							   ((BitmapOrState *) planstate)->bitmapplans,
+			ExplainMemberNodes(((BitmapOrState *) planstate)->bitmapplans,
+							   ((BitmapOrState *) planstate)->nplans,
+							   list_length(((BitmapOr *) plan)->bitmapplans),
 							   ancestors, es);
 			break;
 		case T_SubqueryScan:
@@ -3173,18 +3178,28 @@ show_modifytable_info(ModifyTableState *mtstate, List *ancestors,
  *
  * The ancestors list should already contain the immediate parent of these
  * plans.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
+*
+* nsubnodes indicates the number of items in the planstates array.
+* nplans indicates the original number of subnodes in the Plan, some of these
+* may have been pruned by the run-time pruning code.
  */
 static void
-ExplainMemberNodes(List *plans, PlanState **planstates,
+ExplainMemberNodes(PlanState **planstates, int nsubnodes, int nplans,
 				   List *ancestors, ExplainState *es)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
-	for (j = 0; j < nplans; j++)
+	/*
+	 * The number of subnodes being lower than the number of subplans that was
+	 * specified in the plan means that some subnodes have been ignored per
+	 * instruction for the partition pruning code during the executor
+	 * initialization.  To make this a bit less mysterious, we'll indicate
+	 * here that this has happened.
+	 */
+	if (nsubnodes < nplans)
+		ExplainPropertyInteger("Subplans Pruned", NULL, nplans - nsubnodes, es);
+
+	for (j = 0; j < nsubnodes; j++)
 		ExplainNode(planstates[j], ancestors,
 					"Member", NULL, es);
 }
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index ac94f9f337..1fc19b0624 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -40,6 +40,10 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
 									 bool *isnull,
 									 int maxfieldlen);
 static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
+static void find_subplans_for_params_recurse(PartitionPruneState *prunestate,
+								 PartitionPruningData *pprune,
+								 bool allparams,
+								 Bitmapset **validsubplans);
 
 
 /*
@@ -1293,3 +1297,417 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
 
 	return new_tlist;
 }
+
+/*-------------------------------------------------------------------------
+ * Run-Time Partition Pruning Support.
+ *
+ * The following series of functions exist to support the removal of unneeded
+ * subnodes for queries against partitioned tables.  The supporting functions
+ * here are designed to work with any node type which supports an arbitrary
+ * number of subnodes, e.g. Append, MergeAppend.
+ *
+ * Normally this pruning work is performed by the query planner's partition
+ * pruning code, however, the planner is limited to only being able to prune
+ * away unneeded partitions using quals which compare the partition key to a
+ * value which is known to be Const during planning.  To allow the same
+ * pruning to be performed for values which are only determined during
+ * execution, we must make an additional pruning attempt during execution.
+ *
+ * Here we support pruning using both external and exec Params.  The main
+ * 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.
+ *
+ * For external Params we may be able to prune away unneeded partitions
+ * during executor startup.  This has the added benefit of not having to
+ * initialize the unneeded subnodes at all.  This is useful as it can save
+ * quite a bit of effort during executor startup.
+ *
+ * For exec Params, we must delay pruning until the executor is running.
+ *
+ * Functions:
+ *
+ * ExecSetupPartitionPruneState:
+ *		This must be called by nodes before any partition pruning is
+ *		attempted.  Normally executor startup is a good time. This function
+ *		creates the PartitionPruneState details which are required by each
+ *		of the two pruning functions, details include information about
+ *		how to map the partition index details which are returned by the
+ *		planner's partition prune function into subnode indexes.
+ *
+ * ExecFindInitialMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing only external Params
+ *		to eliminate subnodes.  The function must only be called during
+ *		executor startup for the given node before the subnodes themselves
+ *		are initialized.  Subnodes which are found not to match by this
+ *		function must not be included in the node's list of subnodes as this
+ *		function performs a remap of the partition index to subplan index map
+ *		and the newly created map provides indexes only for subnodes which
+ *		remain after calling this function.
+ *
+ * ExecFindMatchingSubPlans:
+ *		Returns indexes of matching subnodes utilizing all Params to eliminate
+ *		subnodes which can't possibly contain matching tuples.  This function
+ *		can only be called while the executor is running.
+ *-------------------------------------------------------------------------
+ */
+
+/*
+ * ExecSetupPartitionPruneState
+ *		Setup the required data structure which is required for calling
+ *		ExecFindInitialMatchingSubPlans and ExecFindMatchingSubPlans.
+ *
+ * 'partitionpruneinfo' is a List of PartitionPruneInfos as generated by
+ * make_partition_pruneinfo.  Here we build a PartitionPruneContext for each
+ * item in the List.  These contexts can be re-used each time we re-evaulate
+ * which partitions match the pruning steps provided in each
+ * PartitionPruneInfo.
+ */
+PartitionPruneState *
+ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
+{
+	PartitionPruningData *prunedata;
+	PartitionPruneState *prunestate;
+	ListCell   *lc;
+	int			i;
+
+	Assert(partitionpruneinfo != NIL);
+
+	prunestate = (PartitionPruneState *) palloc(sizeof(PartitionPruneState));
+	prunedata = (PartitionPruningData *)
+		palloc(sizeof(PartitionPruningData) *
+			   list_length(partitionpruneinfo));
+
+	/*
+	 * The first item in the array contains the details for the query's target
+	 * partition, so record that as the root of the partition hierarchy.
+	 */
+	prunestate->partprunedata = prunedata;
+	prunestate->num_partprunedata = list_length(partitionpruneinfo);
+	prunestate->extparams = NULL;
+	prunestate->execparams = NULL;
+
+	/*
+	 * Create a sub memory context which we'll use when making calls to the
+	 * query planner's function to determine which partitions will match.  The
+	 * planner is not too careful about freeing memory, so we'll ensure we
+	 * call the function in this context to avoid any memory leaking in the
+	 * executor's memory context.
+	 */
+	prunestate->prune_context = AllocSetContextCreate(CurrentMemoryContext,
+												  "Partition Prune",
+												  ALLOCSET_DEFAULT_SIZES);
+
+	i = 0;
+	foreach(lc, partitionpruneinfo)
+	{
+		PartitionPruneInfo *pinfo = (PartitionPruneInfo *) lfirst(lc);
+		PartitionPruningData *pprune = &prunedata[i];
+		PartitionPruneContext *context = &pprune->context;
+		PartitionDesc partdesc;
+		Relation	rel;
+		PartitionKey partkey;
+		int			partnatts;
+
+		pprune->present_parts = bms_copy(pinfo->present_parts);
+		pprune->subnode_map = palloc(sizeof(int) * pinfo->nparts);
+
+		/*
+		 * We must make a copy of this rather than pointing directly to the
+		 * plan's version as we may end up making modifications to it later.
+		 */
+		memcpy(pprune->subnode_map, pinfo->subnode_map,
+			   sizeof(int) * pinfo->nparts);
+
+		/* We can use the subpart_map verbatim, since we never modify it */
+		pprune->subpart_map = pinfo->subpart_map;
+
+		rel = relation_open(pinfo->reloid, NoLock);
+
+		partkey = RelationGetPartitionKey(rel);
+		partdesc = RelationGetPartitionDesc(rel);
+
+		context->strategy = partkey->strategy;
+		context->partnatts = partnatts = partkey->partnatts;
+
+		context->partopcintype = partkey->partopcintype;
+		context->partopfamily = partkey->partopfamily;
+		context->partcollation = partkey->partcollation;
+		context->partsupfunc = partkey->partsupfunc;
+		context->nparts = pinfo->nparts;
+		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
+
+		context->planstate = planstate;
+		context->safeparams = NULL; /* empty for now */
+
+		pprune->pruning_steps = pinfo->pruning_steps;
+
+		pprune->extparams = bms_copy(pinfo->extparams);
+		pprune->allparams = bms_union(pinfo->extparams, pinfo->execparams);
+
+		/*
+		 * Accumulate the paramids which match the partitioned keys of all
+		 * partitioned tables.
+		 */
+		prunestate->extparams = bms_add_members(prunestate->extparams,
+												pinfo->extparams);
+
+		prunestate->execparams = bms_add_members(prunestate->execparams,
+												 pinfo->execparams);
+
+		relation_close(rel, NoLock);
+
+		i++;
+	}
+
+	/*
+	 * Cache the union of the paramids of both types.  This saves having to
+	 * recalculate it everytime we need to know what they are.
+	 */
+	prunestate->allparams = bms_union(prunestate->extparams,
+									  prunestate->execparams);
+
+	return prunestate;
+}
+
+/*
+ * ExecFindInitialMatchingSubPlans
+ *		Determine which subset of subplan nodes we need to initialize based
+ *		on the details stored in 'prunestate'.  Here we only determine the
+ *		matching partitions using values known during plan startup, which is
+ *		only external Params.  Exec Params will be unknown at this time.  We
+ *		must delay pruning using exec Params until the actual executor run.
+ *
+ * It is expected that callers of this function do so only once during their
+ * init plan.  The caller must only initialize the subnodes which are returned
+ * by this function. The remaining subnodes should be discarded.  Once this
+ * function has been called, future calls to ExecFindMatchingSubPlans will
+ * return its matching subnode indexes assuming that the caller discarded
+ * the original non-matching subnodes.
+ *
+ * This function must only be called if 'prunestate' has any extparams.
+ *
+ * 'nsubnodes' must be passed as the total number of unpruned subnodes.
+ */
+Bitmapset *
+ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate, int nsubnodes)
+{
+	PartitionPruningData *pprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	/*
+	 * Ensure there's actually external params, or we've not been called
+	 * already.
+	 */
+	Assert(!bms_is_empty(prunestate->extparams));
+
+	pprune = prunestate->partprunedata;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(prunestate->prune_context);
+
+	/* Determine which subnodes match the external params */
+	find_subplans_for_params_recurse(prunestate, pprune, false, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(prunestate->prune_context);
+
+	/*
+	 * Record that partition pruning has been performed for external params.
+	 * This partly also serves to ensure we never call this function twice
+	 * with the same input and also so that ExecFindMatchingSubPlans is aware
+	 * that pruning has already been performed for external Params.
+	 */
+	bms_free(prunestate->extparams);
+	prunestate->extparams = NULL;
+
+	/*
+	 * If any subnodes were pruned, we must re-sequence the subnode indexes so
+	 * that ExecFindMatchingSubPlans properly returns the indexes from the
+	 * subnodes which will remain after execution of this function.
+	 */
+	if (bms_num_members(result) < nsubnodes)
+	{
+		int		   *new_subnode_indexes;
+		int			i;
+		int			newidx;
+
+		/*
+		 * First we must build an array which we can use to adjust the
+		 * existing subnode_map so that it contains the new subnode indexes.
+		 */
+		new_subnode_indexes = (int *) palloc(sizeof(int) * nsubnodes);
+		newidx = 0;
+		for (i = 0; i < nsubnodes; i++)
+		{
+			if (bms_is_member(i, result))
+				new_subnode_indexes[i] = newidx++;
+			else
+				new_subnode_indexes[i] = -1;	/* Newly pruned */
+		}
+
+		/*
+		 * Now we can re-sequence each PartitionPruneInfo's subnode_map so
+		 * that they point to the new index of the subnode.
+		 */
+		for (i = 0; i < prunestate->num_partprunedata; i++)
+		{
+			int			nparts;
+			int			j;
+
+			pprune = &prunestate->partprunedata[i];
+			nparts = pprune->context.nparts;
+
+			/*
+			 * We also need to reset the present_parts field so that it only
+			 * contains partition indexes that we actually still have subnodes
+			 * for.  It seems easier to build a fresh one, rather than trying
+			 * to update the existing one.
+			 */
+			bms_free(pprune->present_parts);
+			pprune->present_parts = NULL;
+
+			for (j = 0; j < nparts; j++)
+			{
+				int			oldidx = pprune->subnode_map[j];
+
+				/*
+				 * If this partition existed as a subnode then change the old
+				 * subnode index to the new subnode index.  The new index may
+				 * become -1 if the partition was pruned above, or it may just
+				 * come earlier in the subnode list due to some subnodes being
+				 * removed earlier in the list.
+				 */
+				if (oldidx >= 0)
+				{
+					pprune->subnode_map[j] = new_subnode_indexes[oldidx];
+
+					if (new_subnode_indexes[oldidx] >= 0)
+						pprune->present_parts =
+							bms_add_member(pprune->present_parts, j);
+				}
+			}
+		}
+
+		pfree(new_subnode_indexes);
+	}
+
+	return result;
+}
+
+/*
+ * ExecFindMatchingSubPlans
+ *		Determine which subplans match the the pruning steps detailed in
+ *		'pprune' for the current Param values.
+ *
+ * Here we utilize both external and exec Params for pruning.
+ */
+Bitmapset *
+ExecFindMatchingSubPlans(PartitionPruneState *prunestate)
+{
+	PartitionPruningData *pprune;
+	MemoryContext oldcontext;
+	Bitmapset  *result = NULL;
+
+	pprune = prunestate->partprunedata;
+
+	/*
+	 * Switch to a temp context to avoid leaking memory in the executor's
+	 * memory context.
+	 */
+	oldcontext = MemoryContextSwitchTo(prunestate->prune_context);
+
+	find_subplans_for_params_recurse(prunestate, pprune, true, &result);
+
+	MemoryContextSwitchTo(oldcontext);
+
+	/* Move to the correct memory context */
+	result = bms_copy(result);
+
+	MemoryContextReset(prunestate->prune_context);
+
+	return result;
+}
+
+/*
+ * find_subplans_for_params_recurse
+ *		Recursive worker function for ExecFindMatchingSubPlans and
+ *		ExecFindInitialMatchingSubPlans
+ */
+static void
+find_subplans_for_params_recurse(PartitionPruneState *prunestate,
+								 PartitionPruningData *pprune,
+								 bool allparams,
+								 Bitmapset **validsubplans)
+{
+	PartitionPruneContext *context = &pprune->context;
+	Bitmapset  *partset;
+	Bitmapset  *pruneparams;
+	int			i;
+
+	/* Guard against stack overflow due to overly deep partition hierarchy. */
+	check_stack_depth();
+
+	/*
+	 * Use only external params unless we've been asked to also use exec
+	 * params too.
+	 */
+	if (allparams)
+		pruneparams = pprune->allparams;
+	else
+		pruneparams = pprune->extparams;
+
+	/*
+	 * 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.
+	 */
+	if (!bms_is_empty(pruneparams))
+	{
+		context->safeparams = pruneparams;
+		partset = get_matching_partitions(context,
+										  pprune->pruning_steps);
+	}
+	else
+		partset = pprune->present_parts;
+
+	/* Translate partset into subnode indexes */
+	i = -1;
+	while ((i = bms_next_member(partset, i)) >= 0)
+	{
+		if (pprune->subnode_map[i] >= 0)
+			*validsubplans = bms_add_member(*validsubplans,
+											pprune->subnode_map[i]);
+		else
+		{
+			int			partidx = pprune->subpart_map[i];
+
+			if (partidx != -1)
+				find_subplans_for_params_recurse(prunestate,
+												 &prunestate->partprunedata[partidx],
+												 allparams, validsubplans);
+			else
+			{
+				/*
+				 * This could only happen if clauses used in planning where
+				 * more restrictive than those used here, or if the maps are
+				 * somehow corrupt.
+				 */
+				elog(ERROR, "partition missing from subplans");
+			}
+		}
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9287baaedc..38c5592d32 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -2182,6 +2182,23 @@ _copyPartitionPruneStepCombine(const PartitionPruneStepCombine *from)
 	return newnode;
 }
 
+static PartitionPruneInfo *
+_copyPartitionPruneInfo(const PartitionPruneInfo *from)
+{
+	PartitionPruneInfo *newnode = makeNode(PartitionPruneInfo);
+
+	COPY_SCALAR_FIELD(reloid);
+	COPY_NODE_FIELD(pruning_steps);
+	COPY_BITMAPSET_FIELD(present_parts);
+	COPY_SCALAR_FIELD(nparts);
+	COPY_POINTER_FIELD(subnode_map, from->nparts * sizeof(int));
+	COPY_POINTER_FIELD(subpart_map, from->nparts * sizeof(int));
+	COPY_BITMAPSET_FIELD(extparams);
+	COPY_BITMAPSET_FIELD(execparams);
+
+	return newnode;
+}
+
 /* ****************************************************************
  *						relation.h copy functions
  *
@@ -5121,6 +5138,9 @@ copyObjectImpl(const void *from)
 		case T_PlaceHolderInfo:
 			retval = _copyPlaceHolderInfo(from);
 			break;
+		case T_PartitionPruneInfo:
+			retval = _copyPartitionPruneInfo(from);
+			break;
 
 			/*
 			 * VALUE NODES
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 03a91c3352..f58efcd729 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -1758,6 +1758,30 @@ _outMergeAction(StringInfo str, const MergeAction *node)
 	WRITE_NODE_FIELD(targetList);
 }
 
+static void
+_outPartitionPruneInfo(StringInfo str, const PartitionPruneInfo *node)
+{
+	int			i;
+
+	WRITE_NODE_TYPE("PARTITIONPRUNEINFO");
+
+	WRITE_OID_FIELD(reloid);
+	WRITE_NODE_FIELD(pruning_steps);
+	WRITE_BITMAPSET_FIELD(present_parts);
+	WRITE_INT_FIELD(nparts);
+
+	appendStringInfoString(str, " :subnode_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subnode_map[i]);
+
+	appendStringInfoString(str, " :subpart_map");
+	for (i = 0; i < node->nparts; i++)
+		appendStringInfo(str, " %d", node->subpart_map[i]);
+
+	WRITE_BITMAPSET_FIELD(extparams);
+	WRITE_BITMAPSET_FIELD(execparams);
+}
+
 /*****************************************************************************
  *
  *	Stuff from relation.h.
@@ -3992,6 +4016,9 @@ outNode(StringInfo str, const void *obj)
 			case T_PartitionPruneStepCombine:
 				_outPartitionPruneStepCombine(str, obj);
 				break;
+			case T_PartitionPruneInfo:
+				_outPartitionPruneInfo(str, obj);
+				break;
 			case T_Path:
 				_outPath(str, obj);
 				break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2812dc9646..cf040adcfb 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1373,6 +1373,23 @@ _readMergeAction(void)
 	READ_DONE();
 }
 
+static PartitionPruneInfo *
+_readPartitionPruneInfo(void)
+{
+	READ_LOCALS(PartitionPruneInfo);
+
+	READ_OID_FIELD(reloid);
+	READ_NODE_FIELD(pruning_steps);
+	READ_BITMAPSET_FIELD(present_parts);
+	READ_INT_FIELD(nparts);
+	READ_INT_ARRAY(subnode_map, local_node->nparts);
+	READ_INT_ARRAY(subpart_map, local_node->nparts);
+	READ_BITMAPSET_FIELD(extparams);
+	READ_BITMAPSET_FIELD(execparams);
+
+	READ_DONE();
+}
+
 /*
  *	Stuff from parsenodes.h.
  */
@@ -2645,6 +2662,8 @@ parseNodeString(void)
 		return_value = _readPartitionPruneStepOp();
 	else if (MATCH("PARTITIONPRUNESTEPCOMBINE", 25))
 		return_value = _readPartitionPruneStepCombine();
+	else if (MATCH("PARTITIONPRUNEINFO", 18))
+		return_value = _readPartitionPruneInfo();
 	else if (MATCH("RTE", 3))
 		return_value = _readRangeTblEntry();
 	else if (MATCH("RANGETBLFUNCTION", 16))
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 959ee1643d..ff68dca1e9 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -21,10 +21,12 @@
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
 #include "catalog/pg_type.h"
+#include "executor/executor.h"
 #include "miscadmin.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #include "optimizer/clauses.h"
+#include "optimizer/pathnode.h"
 #include "optimizer/planner.h"
 #include "optimizer/predtest.h"
 #include "optimizer/prep.h"
@@ -135,6 +137,7 @@ 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 bool pull_partkey_params(PartitionPruneInfo *pinfo, List *steps);
 static PruneStepResult *perform_pruning_base_step(PartitionPruneContext *context,
 						  PartitionPruneStepOp *opstep);
 static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *context,
@@ -145,6 +148,181 @@ static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
 						Expr *expr, Datum *value);
 
+/*
+ * make_partition_pruneinfo
+ *		Builds List of PartitionPruneInfos, one for each 'partitioned_rels'.
+ *		These can be used in the executor to allow additional partition
+ *		pruning to take place.
+ *
+ * Here we generate partition pruning steps for 'prunequal' and also build a
+ * data stucture which allows mapping of partition indexes into 'subpaths'
+ * indexes.
+ *
+ * If no Params were found to match the partition key in any of the
+ * 'partitioned_rels', then we return NIL.  In such a case run-time partition
+ * pruning would be useless.
+ */
+List *
+make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal)
+{
+	RelOptInfo *targetpart = NULL;
+	ListCell   *lc;
+	List	   *pinfolist = NIL;
+	int		   *relid_subnode_map;
+	int		   *relid_subpart_map;
+	int			i;
+	bool		gotparam = false;
+
+	/*
+	 * Allocate two arrays to store the 1-based indexes of the 'subpaths' and
+	 * 'partitioned_rels' by relid.
+	 */
+	relid_subnode_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+	relid_subpart_map = palloc0(sizeof(int) * root->simple_rel_array_size);
+
+	i = 1;
+	foreach(lc, subpaths)
+	{
+		Path	   *path = (Path *) lfirst(lc);
+		RelOptInfo *pathrel = path->parent;
+
+		Assert(IS_SIMPLE_REL(pathrel));
+		Assert(pathrel->relid < root->simple_rel_array_size);
+
+		relid_subnode_map[pathrel->relid] = i++;
+	}
+
+	/* Likewise for the partition_rels */
+	i = 1;
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+
+		Assert(rti < root->simple_rel_array_size);
+
+		relid_subpart_map[rti] = i++;
+	}
+
+	/* We now build a PartitionPruneInfo for each partition_rels */
+	foreach(lc, partition_rels)
+	{
+		Index		rti = lfirst_int(lc);
+		RelOptInfo *subpart = find_base_rel(root, rti);
+		PartitionPruneInfo *pinfo;
+		RangeTblEntry *rte;
+		Bitmapset  *present_parts;
+		int			nparts = subpart->nparts;
+		int		   *subnode_map;
+		int		   *subpart_map;
+		List	   *partprunequal;
+		List	   *pruning_steps;
+		bool		contradictory;
+
+		/*
+		 * The first item in the list is the target partitioned relation.  The
+		 * quals belong to this relation, so require no translation.
+		 */
+		if (!targetpart)
+		{
+			targetpart = subpart;
+			partprunequal = prunequal;
+		}
+		else
+		{
+			/*
+			 * For sub-partitioned tables the columns may not be in the same
+			 * order as the parent, so we must translate the prunequal to make
+			 * it compatible with this relation.
+			 */
+			partprunequal = (List *)
+				adjust_appendrel_attrs_multilevel(root,
+												  (Node *) prunequal,
+												  subpart->relids,
+												  targetpart->relids);
+		}
+
+		pruning_steps = gen_partprune_steps(subpart, partprunequal,
+											&contradictory);
+
+		if (contradictory)
+		{
+			/*
+			 * This shouldn't happen as the planner should have detected this
+			 * earlier. However, we do use additional quals from parameterized
+			 * paths here. These do only compare Params to the partition key,
+			 * so this shouldn't cause the discovery of any new qual
+			 * contradictions that were not previously discovered as the Param
+			 * values are unknown during planning.  Anyway, we'd better do
+			 * something sane here, so let's just disable run-time pruning.
+			 */
+			return NIL;
+		}
+
+		subnode_map = (int *) palloc(nparts * sizeof(int));
+		subpart_map = (int *) palloc(nparts * sizeof(int));
+		present_parts = NULL;
+
+		/*
+		 * Loop over each partition of the partitioned rel and record the
+		 * subpath index for each.  Any partitions which are not present in
+		 * the subpaths List will be set to -1, and any sub-partitioned table
+		 * which is not present will also be set to -1.
+		 */
+		for (i = 0; i < nparts; i++)
+		{
+			RelOptInfo *partrel = subpart->part_rels[i];
+			int			subnodeidx = relid_subnode_map[partrel->relid] - 1;
+			int			subpartidx = relid_subpart_map[partrel->relid] - 1;
+
+			subnode_map[i] = subnodeidx;
+			subpart_map[i] = subpartidx;
+
+			/*
+			 * Record the indexes of all the partition indexes that we have
+			 * subnodes or subparts for.  This allows an optimization to skip
+			 * attempting any run-time pruning when no Params are found
+			 * matching the partition key at this level.
+			 */
+			if (subnodeidx >= 0 || subpartidx >= 0)
+				present_parts = bms_add_member(present_parts, i);
+		}
+
+		rte = root->simple_rte_array[subpart->relid];
+
+		pinfo = makeNode(PartitionPruneInfo);
+		pinfo->reloid = rte->relid;
+		pinfo->pruning_steps = pruning_steps;
+		pinfo->present_parts = present_parts;
+		pinfo->nparts = nparts;
+		pinfo->extparams = NULL;
+		pinfo->execparams = NULL;
+		pinfo->subnode_map = subnode_map;
+		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.
+		 */
+		gotparam |= pull_partkey_params(pinfo, pruning_steps);
+
+		pinfolist = lappend(pinfolist, pinfo);
+	}
+
+	pfree(relid_subnode_map);
+	pfree(relid_subpart_map);
+
+	if (gotparam)
+		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.
+	 */
+	return NIL;
+}
 
 /*
  * gen_partprune_steps
@@ -240,6 +418,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	context.nparts = rel->nparts;
 	context.boundinfo = rel->boundinfo;
 
+	/* Not valid when being called from the planner */
+	context.planstate = NULL;
+	context.safeparams = NULL;
+
 	/* Actual pruning happens here. */
 	partindexes = get_matching_partitions(&context, pruning_steps);
 
@@ -2473,6 +2655,57 @@ get_matching_range_bounds(PartitionPruneContext *context,
 	return result;
 }
 
+/*
+ * pull_partkey_params
+ *		Loop through each pruning step and record each external and exec
+ *		Params being compared to the partition keys.
+ */
+static bool
+pull_partkey_params(PartitionPruneInfo *pinfo, List *steps)
+{
+	ListCell   *lc;
+	bool		gotone = false;
+
+	foreach(lc, steps)
+	{
+		PartitionPruneStepOp *stepop = lfirst(lc);
+		ListCell   *lc2;
+
+		if (!IsA(stepop, PartitionPruneStepOp))
+			continue;
+
+		foreach(lc2, stepop->exprs)
+		{
+			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;
+
+					default:
+						elog(ERROR, "unrecognized paramkind: %d",
+							 (int) param->paramkind);
+						break;
+				}
+				gotone = true;
+			}
+		}
+	}
+
+	return gotone;
+}
+
 /*
  * perform_pruning_base_step
  *		Determines the indexes of datums that satisfy conditions specified in
@@ -2774,6 +3007,29 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			*value = ((Const *) expr)->constvalue;
 			return true;
 
+		case T_Param:
+
+			/*
+			 * 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;
+				bool		isNull;
+
+				exprstate = ExecInitExpr(expr, context->planstate);
+
+				*value = ExecEvalExprSwitchContext(exprstate,
+												   context->planstate->ps_ExprContext,
+												   &isNull);
+				if (isNull)
+					return false;
+
+				return true;
+			}
+
 		default:
 			break;
 	}
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 63c883093e..b2daf24c41 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -17,6 +17,7 @@
 #include "nodes/execnodes.h"
 #include "nodes/parsenodes.h"
 #include "nodes/plannodes.h"
+#include "partitioning/partprune.h"
 
 /*-----------------------
  * PartitionDispatch - information about one partitioned table in a partition
@@ -108,6 +109,77 @@ typedef struct PartitionTupleRouting
 	TupleTableSlot *root_tuple_slot;
 } PartitionTupleRouting;
 
+/*-----------------------
+ * PartitionPruningData - Encapsulates all information required to support
+ * elimination of partitions in node types which support arbitrary Lists of
+ * subplans.  Information stored here allows the planner's partition pruning
+ * functions to be called and the return value of partition indexes translated
+ * into the subpath indexes of node types such as Append, thus allowing us to
+ * bypass certain subnodes when we have proofs that indicate that no tuple
+ * matching the 'pruning_steps' will be found within.
+ *
+ * subnode_map					An array containing the subnode index which
+ *								matches this partition index, or -1 if the
+ *								subnode has been pruned already.
+ * subpart_map					An array containing the offset into the
+ *								'partprunedata' array in PartitionPruning, or
+ *								-1 if there is no such element in that array.
+ * present_parts				A Bitmapset of the partition index that we have
+ *								subnodes mapped for.
+ * context						Contains the context details required to call
+ *								the partition pruning code.
+ * pruning_steps				Contains a list of PartitionPruneStep used to
+ *								perform the actual pruning.
+ * extparams					Contains paramids of external params found
+ *								matching partition keys in 'pruning_steps'.
+ * allparams					As 'extparams' but also including exec params.
+ *-----------------------
+ */
+typedef struct PartitionPruningData
+{
+	int		   *subnode_map;
+	int		   *subpart_map;
+	Bitmapset  *present_parts;
+	PartitionPruneContext context;
+	List	   *pruning_steps;
+	Bitmapset  *extparams;
+	Bitmapset  *allparams;
+} PartitionPruningData;
+
+/*-----------------------
+ * PartitionPruneState - State object required for executor nodes to perform
+ * partition pruning elimination of their subnodes.  This encapsulates a
+ * flattened hierarchy of PartitionPruningData structs and also stores all
+ * paramids which were found to match the partition keys of each partition.
+ * This struct can be attached to node types which support arbitrary Lists of
+ * subnodes containing partitions to allow subnodes to be eliminated due to
+ * the clauses being unable to match to any tuple that the subnode could
+ * possibly produce.
+ *
+ * partprunedata		Array of PartitionPruningData for the node's target
+ *						partitioned relation. First element contains the
+ *						details for the target partitioned table.
+ * num_partprunedata	Number of items in 'partprunedata' array.
+ * 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
+ *						partition key in each of the contained
+ *						PartitionPruningData structs.
+ * execparams			As above but for PARAM_EXEC.
+ * allparams			Union of 'extparams' and 'execparams', saved to avoid
+ *						recalculation.
+ *-----------------------
+ */
+typedef struct PartitionPruneState
+{
+	PartitionPruningData *partprunedata;
+	int			num_partprunedata;
+	MemoryContext prune_context;
+	Bitmapset  *extparams;
+	Bitmapset  *execparams;
+	Bitmapset  *allparams;
+} PartitionPruneState;
+
 extern PartitionTupleRouting *ExecSetupPartitionTupleRouting(ModifyTableState *mtstate,
 							   Relation rel);
 extern int ExecFindPartition(ResultRelInfo *resultRelInfo,
@@ -133,5 +205,10 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 						  TupleTableSlot **p_my_slot);
 extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
 						PartitionTupleRouting *proute);
+extern PartitionPruneState *ExecSetupPartitionPruneState(PlanState *planstate,
+						  List *partitionpruneinfo);
+extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
+extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
+								int nsubnodes);
 
 #endif							/* EXECPARTITION_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 4fc2de7184..defdbae507 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -196,6 +196,7 @@ typedef enum NodeTag
 	T_PartitionPruneStep,
 	T_PartitionPruneStepOp,
 	T_PartitionPruneStepCombine,
+	T_PartitionPruneInfo,
 
 	/*
 	 * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h
index ff5c4ff8e4..f90aa7b2a1 100644
--- a/src/include/nodes/primnodes.h
+++ b/src/include/nodes/primnodes.h
@@ -1581,4 +1581,27 @@ typedef struct PartitionPruneStepCombine
 	List	   *source_stepids;
 } PartitionPruneStepCombine;
 
+/*----------
+ * PartitionPruneInfo - Details required to allow the executor to prune
+ * partitions.
+ *
+ * Here we store mapping details to allow translation of a partitioned table's
+ * index into subnode indexes for node types which support arbitrary numbers
+ * of sub nodes, such as Append.
+ *----------
+ */
+typedef struct PartitionPruneInfo
+{
+	NodeTag		type;
+	Oid			reloid;			/* Oid of partition rel */
+	List	   *pruning_steps;	/* List of PartitionPruneStep */
+	Bitmapset  *present_parts;	/* Indexes of all partitions which subnodes
+								 * are present for. */
+	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 */
+	Bitmapset  *extparams;		/* All external paramids seen in prunesteps */
+	Bitmapset  *execparams;		/* All exec paramids seen in prunesteps */
+} PartitionPruneInfo;
+
 #endif							/* PRIMNODES_H */
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index 52fadc7caf..f68b6c9935 100644
--- a/src/include/partitioning/partprune.h
+++ b/src/include/partitioning/partprune.h
@@ -37,9 +37,23 @@ typedef struct PartitionPruneContext
 
 	/* Partition boundary info */
 	PartitionBoundInfo boundinfo;
+
+	/*
+	 * Can be set when the context is used from the executor to allow params
+	 * found matching the partition key to be evaulated.
+	 */
+	PlanState  *planstate;
+
+	/*
+	 * Parameters that are safe to be used for partition pruning. execparams
+	 * are not safe to use until the executor is running.
+	 */
+	Bitmapset  *safeparams;
 } PartitionPruneContext;
 
 
+extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
+						 List *subpaths, List *prunequal);
 extern Relids prune_append_rel_partitions(RelOptInfo *rel);
 extern Bitmapset *get_matching_partitions(PartitionPruneContext *context,
 						List *pruning_steps);
-- 
2.16.2.windows.1

v23-0002-Add-bms_prev_member-function.patchapplication/octet-stream; name=v23-0002-Add-bms_prev_member-function.patchDownload
From b4f8965b4be09b4d71e5fa2aa68de4e92e0d275d Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sat, 7 Apr 2018 20:24:27 +1200
Subject: [PATCH v23 2/4] Add bms_prev_member function

This works very much like the existing bms_last_member function, only it
traverses through the Bitmapset in the opposite direction from the most
significant bit down to the least significant bit.  A special prevbit value of
-1 may be used to have the function determine the most significant bit.  This
is useful for starting a loop.  When there are no members less than prevbit,
the function returns -2 to indicate there are no more members.
---
 src/backend/nodes/bitmapset.c | 95 +++++++++++++++++++++++++++++++++++++++++++
 src/include/nodes/bitmapset.h |  1 +
 2 files changed, 96 insertions(+)

diff --git a/src/backend/nodes/bitmapset.c b/src/backend/nodes/bitmapset.c
index edcd19a4fd..9341bf579e 100644
--- a/src/backend/nodes/bitmapset.c
+++ b/src/backend/nodes/bitmapset.c
@@ -58,6 +58,9 @@
  * rightmost_one_pos[x] gives the bit number (0-7) of the rightmost one bit
  * in a nonzero byte value x.  The entry for x=0 is never used.
  *
+ * leftmost_one_pos[x] gives the bit number (0-7) of the leftmost one bit in a
+ * nonzero byte value x.  The entry for x=0 is never used.
+ *
  * number_of_ones[x] gives the number of one-bits (0-8) in a byte value x.
  *
  * We could make these tables larger and reduce the number of iterations
@@ -84,6 +87,25 @@ static const uint8 rightmost_one_pos[256] = {
 	4, 0, 1, 0, 2, 0, 1, 0, 3, 0, 1, 0, 2, 0, 1, 0
 };
 
+static const uint8 leftmost_one_pos[256] = {
+	0, 0, 1, 1, 2, 2, 2, 2, 3, 3, 3, 3, 3, 3, 3, 3,
+	4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7,
+	7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+};
+
 static const uint8 number_of_ones[256] = {
 	0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
 	1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
@@ -1088,6 +1110,79 @@ bms_next_member(const Bitmapset *a, int prevbit)
 	return -2;
 }
 
+/*
+ * bms_prev_member - find prev member of a set
+ *
+ * Returns largest member less than "prevbit", or -2 if there is none.
+ * "prevbit" must NOT be more than one above the highest possible bit that can
+ * be set at the Bitmapset at its current size.
+ *
+ * To ease finding the highest set bit for the initial loop, the special
+ * prevbit value of -1 can be passed to have the function find the highest
+ * valued member in the set.
+ *
+ * This is intended as support for iterating through the members of a set in
+ * reverse.  The typical pattern is
+ *
+ *			x = -1;
+ *			while ((x = bms_prev_member(inputset, x)) >= 0)
+ *				process member x;
+ *
+ * Notice that when there are no more members, we return -2, not -1 as you
+ * might expect.  The rationale for that is to allow distinguishing the
+ * loop-not-started state (x == -1) from the loop-completed state (x == -2).
+ * It makes no difference in simple loop usage, but complex iteration logic
+ * might need such an ability.
+ */
+
+int
+bms_prev_member(const Bitmapset *a, int prevbit)
+{
+	int			wordnum;
+	int			ushiftbits;
+	bitmapword	mask;
+
+	/*
+	 * If set is NULL or if there are no more bits to the right then we've
+	 * nothing to do.
+	 */
+	if (a == NULL || prevbit == 0)
+		return -2;
+
+	/* transform -1 to the highest possible bit we could have set */
+	if (prevbit == -1)
+		prevbit = a->nwords * BITS_PER_BITMAPWORD - 1;
+	else
+		prevbit--;
+
+	ushiftbits = BITS_PER_BITMAPWORD - (BITNUM(prevbit) + 1);
+	mask = (~(bitmapword) 0) >> ushiftbits;
+	for (wordnum = WORDNUM(prevbit); wordnum >= 0; wordnum--)
+	{
+		bitmapword	w = a->words[wordnum];
+
+		/* mask out bits left of prevbit */
+		w &= mask;
+
+		if (w != 0)
+		{
+			int			result;
+			int			shift = 24;
+			result = wordnum * BITS_PER_BITMAPWORD;
+
+			while ((w >> shift) == 0)
+				shift -= 8;
+
+			result += shift + leftmost_one_pos[(w >> shift) & 255];
+			return result;
+		}
+
+		/* in subsequent words, consider all bits */
+		mask = (~(bitmapword) 0);
+	}
+	return -2;
+}
+
 /*
  * bms_hash_value - compute a hash key for a Bitmapset
  *
diff --git a/src/include/nodes/bitmapset.h b/src/include/nodes/bitmapset.h
index 67e8920f65..b6f1a9e6e5 100644
--- a/src/include/nodes/bitmapset.h
+++ b/src/include/nodes/bitmapset.h
@@ -99,6 +99,7 @@ extern Bitmapset *bms_join(Bitmapset *a, Bitmapset *b);
 /* support for iterating through the integer elements of a set: */
 extern int	bms_first_member(Bitmapset *a);
 extern int	bms_next_member(const Bitmapset *a, int prevbit);
+extern int	bms_prev_member(const Bitmapset *a, int prevbit);
 
 /* support for hashtables using Bitmapsets as keys: */
 extern uint32 bms_hash_value(const Bitmapset *a);
-- 
2.16.2.windows.1

v23-0003-Allow-unneeded-Append-subnodes-to-be-pruned-at-e.patchapplication/octet-stream; name=v23-0003-Allow-unneeded-Append-subnodes-to-be-pruned-at-e.patchDownload
From 3b0a12201e82a3758293af8f360cd92874cf9baa Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sat, 7 Apr 2018 20:25:09 +1200
Subject: [PATCH v23 3/4] Allow unneeded Append subnodes to be pruned at 
 execution

Support partition pruning of Append subnodes which cannot possibly contain any
matching tuples.  Normally the partition elimination is done during planning,
however, it's only possible to prune partitions pruning planning when the
value in the WHERE clause is a known Const to the planner.  This commit allows
Appends to further prune away unneeded subnodes during execution by evaluating
Params to determine the minimum set of subnodes that can possibly match.  Here
we support more than just simple Params in WHERE clauses. Support includes:

1. Parameterized Nested Loop Joins: The parameter from the outer side of the
   join can be used to determine the minimum set of inner side partitions to
   scan.

2. Initplans: Once an initplan has been executed we can then determine which
   partitions match the value from the initplan.

Partition pruning is performed in two ways.  When Params external to the plan
are found to match the partition key we attempt to prune away unneeded Append
subplans during the initialization of the executor.  This allows us to bypass
the initialization of non-matching subplans meaning they won't appear in the
EXPLAIN or EXPLAIN ANALYZE output.

For parameters whose value is only known during the actual execution then the
pruning of these subplans must wait.  Subplans which are eliminated during
this stage of pruning are still visible in the EXPLAIN output.  In order to
determine if pruning has actually taken place, the EXPLAIN ANALYZE must be
viewed.  If a certain Append subplan was never executed due to the elimination
of the partition then the execution timing area will state "(never executed)".
Whereas, if, for example in the case of parameterized nested loops, the number
of loops stated in the EXPLAIN ANALYZE output for certain subplans may appear
lower than others due to the subplan having been scanned fewer times.  This is
due to the list of matching subnodes having to be evaluated whenever a
parameter which was found to match the partition key changes.
---
 src/backend/executor/nodeAppend.c             |  268 ++++--
 src/backend/nodes/copyfuncs.c                 |    1 +
 src/backend/nodes/nodeFuncs.c                 |   28 +-
 src/backend/nodes/outfuncs.c                  |    1 +
 src/backend/nodes/readfuncs.c                 |    1 +
 src/backend/optimizer/path/allpaths.c         |   12 +-
 src/backend/optimizer/path/joinrels.c         |    2 +-
 src/backend/optimizer/plan/createplan.c       |   44 +-
 src/backend/optimizer/plan/planner.c          |    8 +-
 src/backend/optimizer/prep/prepunion.c        |    6 +-
 src/backend/optimizer/util/pathnode.c         |   26 +-
 src/include/nodes/execnodes.h                 |   12 +-
 src/include/nodes/plannodes.h                 |    5 +
 src/include/optimizer/pathnode.h              |    2 +-
 src/test/regress/expected/partition_prune.out | 1135 +++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  344 ++++++++
 16 files changed, 1804 insertions(+), 91 deletions(-)

diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index dcbf4d68aa..b135b61324 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -58,6 +58,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeAppend.h"
 #include "miscadmin.h"
 
@@ -77,11 +78,13 @@ struct ParallelAppendState
 };
 
 #define INVALID_SUBPLAN_INDEX		-1
+#define NO_MATCHING_SUBPLANS		-2
 
 static TupleTableSlot *ExecAppend(PlanState *pstate);
 static bool choose_next_subplan_locally(AppendState *node);
 static bool choose_next_subplan_for_leader(AppendState *node);
 static bool choose_next_subplan_for_worker(AppendState *node);
+static void mark_invalid_subplans_as_finished(AppendState *node);
 
 /* ----------------------------------------------------------------
  *		ExecInitAppend
@@ -99,8 +102,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 {
 	AppendState *appendstate = makeNode(AppendState);
 	PlanState **appendplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -112,55 +117,117 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->appendplans);
-
-	appendplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
-	appendstate->appendplans = appendplanstates;
-	appendstate->as_nplans = nplans;
+
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
+
+	/* If run-time partition pruning is enabled, then set that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruneState *prunestate;
+
+		ExecAssignExprContext(estate, &appendstate->ps);
+
+		prunestate = ExecSetupPartitionPruneState(&appendstate->ps,
+												  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away Append subplans now.
+		 */
+		if (!bms_is_empty(prunestate->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(prunestate,
+															list_length(node->appendplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires an Append to have at least one subplan in
+			 * order for it to properly determine the Vars in that subplan's
+			 * targetlist.  We sidestep this issue by just initializing the
+			 * first subplan and setting as_whichplan to NO_MATCHING_SUBPLANS
+			 * to indicate that we don't need to scan any subnodes.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				appendstate->as_whichplan = NO_MATCHING_SUBPLANS;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->appendplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there's no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 */
+		if (bms_is_empty(prunestate->execparams))
+			appendstate->as_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+
+		appendstate->as_prune_state = prunestate;
+
+	}
+	else
+	{
+		nplans = list_length(node->appendplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid, they must also all be initialized.
+		 */
+		appendstate->as_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		appendstate->as_prune_state = NULL;
+	}
 
 	/*
 	 * Initialize result tuple type and slot.
 	 */
 	ExecInitResultTupleSlotTL(estate, &appendstate->ps);
 
+	appendplanstates = (PlanState **) palloc(nplans *
+											 sizeof(PlanState *));
+
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "appendplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the appendplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->appendplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		appendplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			appendplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
+	appendstate->appendplans = appendplanstates;
+	appendstate->as_nplans = nplans;
+
 	/*
 	 * Miscellaneous initialization
-	 *
-	 * Append plans don't have expression contexts because they never call
-	 * ExecQual or ExecProject.
 	 */
-	appendstate->ps.ps_ProjInfo = NULL;
 
-	/*
-	 * Parallel-aware append plans must choose the first subplan to execute by
-	 * looking at shared memory, but non-parallel-aware append plans can
-	 * always start with the first subplan.
-	 */
-	appendstate->as_whichplan =
-		appendstate->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	appendstate->ps.ps_ProjInfo = NULL;
 
 	/* For parallel query, this will be overridden later. */
 	appendstate->choose_next_subplan = choose_next_subplan_locally;
@@ -179,10 +246,20 @@ ExecAppend(PlanState *pstate)
 {
 	AppendState *node = castNode(AppendState, pstate);
 
-	/* If no subplan has been chosen, we must choose one before proceeding. */
-	if (node->as_whichplan == INVALID_SUBPLAN_INDEX &&
-		!node->choose_next_subplan(node))
-		return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+	if (node->as_whichplan < 0)
+	{
+		/*
+		 * If no subplan has been chosen, we must choose one before
+		 * proceeding.
+		 */
+		if (node->as_whichplan == INVALID_SUBPLAN_INDEX &&
+			!node->choose_next_subplan(node))
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Nothing to do if there are no matching subplans */
+		else if (node->as_whichplan == NO_MATCHING_SUBPLANS)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+	}
 
 	for (;;)
 	{
@@ -251,6 +328,19 @@ ExecReScanAppend(AppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->as_prune_state &&
+		bms_overlap(node->ps.chgParam,
+					node->as_prune_state->execparams))
+	{
+		bms_free(node->as_valid_subplans);
+		node->as_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->as_nplans; i++)
 	{
 		PlanState  *subnode = node->appendplans[i];
@@ -270,8 +360,8 @@ ExecReScanAppend(AppendState *node)
 			ExecReScan(subnode);
 	}
 
-	node->as_whichplan =
-		node->ps.plan->parallel_aware ? INVALID_SUBPLAN_INDEX : 0;
+	/* Let choose_next_subplan_* function handle setting the first subplan */
+	node->as_whichplan = INVALID_SUBPLAN_INDEX;
 }
 
 /* ----------------------------------------------------------------
@@ -360,29 +450,39 @@ static bool
 choose_next_subplan_locally(AppendState *node)
 {
 	int			whichplan = node->as_whichplan;
+	int			nextplan;
 
-	if (ScanDirectionIsForward(node->ps.state->es_direction))
+	/* We should never be called when there are no subplans */
+	Assert(whichplan != NO_MATCHING_SUBPLANS);
+
+	/*
+	 * If first call then have the bms member function choose the first valid
+	 * subplan by initializing whichplan to -1.  If there happen to be no
+	 * valid subplans then the bms member function will handle that by
+	 * returning a negative number which will allow us to exit returning a
+	 * false value.
+	 */
+	if (whichplan == INVALID_SUBPLAN_INDEX)
 	{
-		/*
-		 * We won't normally see INVALID_SUBPLAN_INDEX in this case, but we
-		 * might if a plan intended to be run in parallel ends up being run
-		 * serially.
-		 */
-		if (whichplan == INVALID_SUBPLAN_INDEX)
-			node->as_whichplan = 0;
-		else
-		{
-			if (whichplan >= node->as_nplans - 1)
-				return false;
-			node->as_whichplan++;
-		}
+		if (node->as_valid_subplans == NULL)
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->as_prune_state);
+
+		whichplan = -1;
 	}
+
+	/* Ensure whichplan is within the expected range */
+	Assert(whichplan >= -1 && whichplan <= node->as_nplans);
+
+	if (ScanDirectionIsForward(node->ps.state->es_direction))
+		nextplan = bms_next_member(node->as_valid_subplans, whichplan);
 	else
-	{
-		if (whichplan <= 0)
-			return false;
-		node->as_whichplan--;
-	}
+		nextplan = bms_prev_member(node->as_valid_subplans, whichplan);
+
+	if (nextplan < 0)
+		return false;
+
+	node->as_whichplan = nextplan;
 
 	return true;
 }
@@ -404,6 +504,9 @@ choose_next_subplan_for_leader(AppendState *node)
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
 
+	/* We should never be called when there are no subplans */
+	Assert(node->as_whichplan != NO_MATCHING_SUBPLANS);
+
 	LWLockAcquire(&pstate->pa_lock, LW_EXCLUSIVE);
 
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
@@ -415,6 +518,23 @@ choose_next_subplan_for_leader(AppendState *node)
 	{
 		/* Start with last subplan. */
 		node->as_whichplan = node->as_nplans - 1;
+
+		/*
+		 * If we've yet to determine the valid subplans for these parameters
+		 * then do so now.  If run-time pruning is disabled then the valid
+		 * subplans will always be set to all subplans.
+		 */
+		if (node->as_valid_subplans == NULL)
+		{
+			node->as_valid_subplans =
+				ExecFindMatchingSubPlans(node->as_prune_state);
+
+			/*
+			 * Mark each invalid plan as finished to allow the loop below to
+			 * select the first valid subplan.
+			 */
+			mark_invalid_subplans_as_finished(node);
+		}
 	}
 
 	/* Loop until we find a subplan to execute. */
@@ -461,12 +581,27 @@ choose_next_subplan_for_worker(AppendState *node)
 	/* Backward scan is not supported by parallel-aware plans */
 	Assert(ScanDirectionIsForward(node->ps.state->es_direction));
 
+	/* We should never be called when there are no subplans */
+	Assert(node->as_whichplan != NO_MATCHING_SUBPLANS);
+
 	LWLockAcquire(&pstate->pa_lock, LW_EXCLUSIVE);
 
 	/* Mark just-completed subplan as finished. */
 	if (node->as_whichplan != INVALID_SUBPLAN_INDEX)
 		node->as_pstate->pa_finished[node->as_whichplan] = true;
 
+	/*
+	 * If we've yet to determine the valid subplans for these parameters then
+	 * do so now.  If run-time pruning is disabled then the valid subplans
+	 * will always be set to all subplans.
+	 */
+	else if (node->as_valid_subplans == NULL)
+	{
+		node->as_valid_subplans =
+			ExecFindMatchingSubPlans(node->as_prune_state);
+		mark_invalid_subplans_as_finished(node);
+	}
+
 	/* If all the plans are already done, we have nothing to do */
 	if (pstate->pa_next_plan == INVALID_SUBPLAN_INDEX)
 	{
@@ -532,3 +667,34 @@ choose_next_subplan_for_worker(AppendState *node)
 
 	return true;
 }
+
+/*
+ * mark_invalid_subplans_as_finished
+ *		Marks the ParallelAppendState's pa_finished as true for each invalid
+ *		subplan.
+ *
+ * This function should only be called for parallel Append with run-time
+ * pruning enabled.
+ */
+static void
+mark_invalid_subplans_as_finished(AppendState *node)
+{
+	int			i;
+
+	/* Only valid to call this while in parallel Append mode */
+	Assert(node->as_pstate);
+
+	/* Shouldn't have been called when run-time pruning is not enabled */
+	Assert(node->as_prune_state);
+
+	/* Nothing to do if all plans are valid */
+	if (bms_num_members(node->as_valid_subplans) == node->as_nplans)
+		return;
+
+	/* Mark all non-valid plans as finished */
+	for (i = 0; i < node->as_nplans; i++)
+	{
+		if (!bms_is_member(i, node->as_valid_subplans))
+			node->as_pstate->pa_finished[i] = true;
+	}
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 38c5592d32..765f290bc7 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -248,6 +248,7 @@ _copyAppend(const Append *from)
 	COPY_NODE_FIELD(partitioned_rels);
 	COPY_NODE_FIELD(appendplans);
 	COPY_SCALAR_FIELD(first_partial_plan);
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index 51c418778a..4c309d236a 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -30,7 +30,7 @@ static int	leftmostLoc(int loc1, int loc2);
 static bool fix_opfuncids_walker(Node *node, void *context);
 static bool planstate_walk_subplans(List *plans, bool (*walker) (),
 									void *context);
-static bool planstate_walk_members(List *plans, PlanState **planstates,
+static bool planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context);
 
 
@@ -3806,32 +3806,32 @@ planstate_tree_walker(PlanState *planstate,
 	switch (nodeTag(plan))
 	{
 		case T_ModifyTable:
-			if (planstate_walk_members(((ModifyTable *) plan)->plans,
-									   ((ModifyTableState *) planstate)->mt_plans,
+			if (planstate_walk_members(((ModifyTableState *) planstate)->mt_plans,
+									   ((ModifyTableState *) planstate)->mt_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_Append:
-			if (planstate_walk_members(((Append *) plan)->appendplans,
-									   ((AppendState *) planstate)->appendplans,
+			if (planstate_walk_members(((AppendState *) planstate)->appendplans,
+									   ((AppendState *) planstate)->as_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_MergeAppend:
-			if (planstate_walk_members(((MergeAppend *) plan)->mergeplans,
-									   ((MergeAppendState *) planstate)->mergeplans,
+			if (planstate_walk_members(((MergeAppendState *) planstate)->mergeplans,
+									   ((MergeAppendState *) planstate)->ms_nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapAnd:
-			if (planstate_walk_members(((BitmapAnd *) plan)->bitmapplans,
-									   ((BitmapAndState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapAndState *) planstate)->bitmapplans,
+									   ((BitmapAndState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
 		case T_BitmapOr:
-			if (planstate_walk_members(((BitmapOr *) plan)->bitmapplans,
-									   ((BitmapOrState *) planstate)->bitmapplans,
+			if (planstate_walk_members(((BitmapOrState *) planstate)->bitmapplans,
+									   ((BitmapOrState *) planstate)->nplans,
 									   walker, context))
 				return true;
 			break;
@@ -3881,15 +3881,11 @@ planstate_walk_subplans(List *plans,
 /*
  * Walk the constituent plans of a ModifyTable, Append, MergeAppend,
  * BitmapAnd, or BitmapOr node.
- *
- * Note: we don't actually need to examine the Plan list members, but
- * we need the list in order to determine the length of the PlanState array.
  */
 static bool
-planstate_walk_members(List *plans, PlanState **planstates,
+planstate_walk_members(PlanState **planstates, int nplans,
 					   bool (*walker) (), void *context)
 {
-	int			nplans = list_length(plans);
 	int			j;
 
 	for (j = 0; j < nplans; j++)
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index f58efcd729..c274adaacc 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -419,6 +419,7 @@ _outAppend(StringInfo str, const Append *node)
 	WRITE_NODE_FIELD(partitioned_rels);
 	WRITE_NODE_FIELD(appendplans);
 	WRITE_INT_FIELD(first_partial_plan);
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index cf040adcfb..93785e3bdf 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1692,6 +1692,7 @@ _readAppend(void)
 	READ_NODE_FIELD(partitioned_rels);
 	READ_NODE_FIELD(appendplans);
 	READ_INT_FIELD(first_partial_plan);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 65a34a255d..3ba3f87eb7 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1604,7 +1604,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 	 * if we have zero or one live subpath due to constraint exclusion.)
 	 */
 	if (subpaths_valid)
-		add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+		add_path(rel, (Path *) create_append_path(root, rel, subpaths, NIL,
 												  NULL, 0, false,
 												  partitioned_rels, -1));
 
@@ -1646,8 +1646,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 		Assert(parallel_workers > 0);
 
 		/* Generate a partial append path. */
-		appendpath = create_append_path(rel, NIL, partial_subpaths, NULL,
-										parallel_workers,
+		appendpath = create_append_path(root, rel, NIL, partial_subpaths,
+										NULL, parallel_workers,
 										enable_parallel_append,
 										partitioned_rels, -1);
 
@@ -1695,7 +1695,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 							   max_parallel_workers_per_gather);
 		Assert(parallel_workers > 0);
 
-		appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+		appendpath = create_append_path(root, rel, pa_nonpartial_subpaths,
 										pa_partial_subpaths,
 										NULL, parallel_workers, true,
 										partitioned_rels, partial_rows);
@@ -1758,7 +1758,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
 
 		if (subpaths_valid)
 			add_path(rel, (Path *)
-					 create_append_path(rel, subpaths, NIL,
+					 create_append_path(root, rel, subpaths, NIL,
 										required_outer, 0, false,
 										partitioned_rels, -1));
 	}
@@ -2024,7 +2024,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
 	rel->pathlist = NIL;
 	rel->partial_pathlist = NIL;
 
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 3f1c1b3477..2e289d475e 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1230,7 +1230,7 @@ mark_dummy_rel(RelOptInfo *rel)
 	rel->partial_pathlist = NIL;
 
 	/* Set up the dummy path */
-	add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+	add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL, NULL,
 											  0, false, NIL, -1));
 
 	/* Set or update cheapest_total_path and related fields */
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99d0736029..d515b1ea78 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -41,6 +41,7 @@
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
+#include "partitioning/partprune.h"
 #include "utils/lsyscache.h"
 
 
@@ -210,7 +211,7 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
 static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
 				   Index scanrelid, int wtParam);
 static Append *make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels);
+			List *tlist, List *partitioned_rels, List *partpruneinfos);
 static RecursiveUnion *make_recursive_union(List *tlist,
 					 Plan *lefttree,
 					 Plan *righttree,
@@ -1041,6 +1042,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	List	   *tlist = build_path_tlist(root, &best_path->path);
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * The subpaths list could be empty, if every child was proven empty by
@@ -1078,6 +1081,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	/*
 	 * XXX ideally, if there's just one child, we'd not bother to generate an
 	 * Append node but just return the single child.  At the moment this does
@@ -1086,7 +1120,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
 	 */
 
 	plan = make_append(subplans, best_path->first_partial_path,
-					   tlist, best_path->partitioned_rels);
+					   tlist, best_path->partitioned_rels,
+					   partpruneinfos);
 
 	copy_generic_path_info(&plan->plan, (Path *) best_path);
 
@@ -5382,7 +5417,8 @@ make_foreignscan(List *qptlist,
 
 static Append *
 make_append(List *appendplans, int first_partial_plan,
-			List *tlist, List *partitioned_rels)
+			List *tlist, List *partitioned_rels,
+			List *partpruneinfos)
 {
 	Append	   *node = makeNode(Append);
 	Plan	   *plan = &node->plan;
@@ -5394,7 +5430,7 @@ make_append(List *appendplans, int first_partial_plan,
 	node->partitioned_rels = partitioned_rels;
 	node->appendplans = appendplans;
 	node->first_partial_plan = first_partial_plan;
-
+	node->part_prune_infos = partpruneinfos;
 	return node;
 }
 
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 008492bad5..421dc79cc4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3920,7 +3920,8 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
 			paths = lappend(paths, path);
 		}
 		path = (Path *)
-			create_append_path(grouped_rel,
+			create_append_path(root,
+							   grouped_rel,
 							   paths,
 							   NIL,
 							   NULL,
@@ -6852,8 +6853,9 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
 		 * node, which would cause this relation to stop appearing to be a
 		 * dummy rel.)
 		 */
-		rel->pathlist = list_make1(create_append_path(rel, NIL, NIL, NULL,
-													  0, false, NIL, -1));
+		rel->pathlist = list_make1(create_append_path(root, rel, NIL, NIL,
+													  NULL, 0, false, NIL,
+													  -1));
 		rel->partial_pathlist = NIL;
 		set_cheapest(rel);
 		Assert(IS_DUMMY_REL(rel));
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 67e47887fc..2ce4d4496d 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -648,7 +648,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/*
@@ -703,7 +703,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
 		Assert(parallel_workers > 0);
 
 		ppath = (Path *)
-			create_append_path(result_rel, NIL, partial_pathlist,
+			create_append_path(root, result_rel, NIL, partial_pathlist,
 							   NULL, parallel_workers, enable_parallel_append,
 							   NIL, -1);
 		ppath = (Path *)
@@ -814,7 +814,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
 	/*
 	 * Append the child results together.
 	 */
-	path = (Path *) create_append_path(result_rel, pathlist, NIL,
+	path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
 									   NULL, 0, false, NIL, -1);
 
 	/* Identify the grouping semantics */
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 416b3f9578..bd9442c22d 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1210,7 +1210,8 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
  * Note that we must handle subpaths = NIL, representing a dummy access path.
  */
 AppendPath *
-create_append_path(RelOptInfo *rel,
+create_append_path(PlannerInfo *root,
+				   RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
@@ -1224,8 +1225,25 @@ create_append_path(RelOptInfo *rel,
 	pathnode->path.pathtype = T_Append;
 	pathnode->path.parent = rel;
 	pathnode->path.pathtarget = rel->reltarget;
-	pathnode->path.param_info = get_appendrel_parampathinfo(rel,
-															required_outer);
+
+	/*
+	 * When generating an Append path for a partitioned table, there may be
+	 * parameters that are useful so we can eliminate certain partitions
+	 * during execution.  Here we'll go all the way and fully populate the
+	 * parameter info data as we do for normal base relations.  However, we
+	 * need only bother doing this for RELOPT_BASEREL rels, as
+	 * RELOPT_OTHER_MEMBER_REL's Append paths are merged into the base rel's
+	 * Append subpaths.  It would do no harm to do this, we just avoid it to
+	 * save wasting effort.
+	 */
+	if (partitioned_rels != NIL && root && rel->reloptkind == RELOPT_BASEREL)
+		pathnode->path.param_info = get_baserel_parampathinfo(root,
+															  rel,
+															  required_outer);
+	else
+		pathnode->path.param_info = get_appendrel_parampathinfo(rel,
+																required_outer);
+
 	pathnode->path.parallel_aware = parallel_aware;
 	pathnode->path.parallel_safe = rel->consider_parallel;
 	pathnode->path.parallel_workers = parallel_workers;
@@ -3574,7 +3592,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
 					i++;
 				}
 				return (Path *)
-					create_append_path(rel, childpaths, partialpaths,
+					create_append_path(root, rel, childpaths, partialpaths,
 									   required_outer,
 									   apath->path.parallel_workers,
 									   apath->path.parallel_aware,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 538e679cdf..efd0752389 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1120,8 +1120,13 @@ typedef struct ModifyTableState
 /* ----------------
  *	 AppendState information
  *
- *		nplans			how many plans are in the array
- *		whichplan		which plan is being executed (0 .. n-1)
+ *		nplans				how many plans are in the array
+ *		whichplan			which plan is being executed (0 .. n-1), or a
+ *							special negative value. See nodeAppend.c.
+ *		pruningstate		details required to allow partitions to be
+ *							eliminated from the scan, or NULL if not possible.
+ *		valid_subplans		for runtime pruning, valid appendplans indexes to
+ *							scan.
  * ----------------
  */
 
@@ -1129,6 +1134,7 @@ struct AppendState;
 typedef struct AppendState AppendState;
 struct ParallelAppendState;
 typedef struct ParallelAppendState ParallelAppendState;
+struct PartitionPruneState;
 
 struct AppendState
 {
@@ -1138,6 +1144,8 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	struct PartitionPruneState *as_prune_state;
+	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
 };
 
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 0a797f0a05..c3e5c2c79f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -256,6 +256,11 @@ typedef struct Append
 	List	   *partitioned_rels;
 	List	   *appendplans;
 	int			first_partial_plan;
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } Append;
 
 /* ----------------
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 895bf6959d..4f65686d9b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -64,7 +64,7 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
 					  List *bitmapquals);
 extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
 					List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel,
+extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
 				   List *subpaths, List *partial_subpaths,
 				   Relids required_outer,
 				   int parallel_workers, bool parallel_aware,
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 69d541eff4..99d94d9a9d 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -1331,3 +1331,1138 @@ explain (costs off) select * from rparted_by_int2 where a > 100000000000000;
 (3 rows)
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, rp, coll_pruning_multi, like_op_noprune, lparted_by_int2, rparted_by_int2;
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+execute ab_q1 (1, 8, 3);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(8 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+                       QUERY PLAN                        
+---------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 3
+   ->  Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+   ->  Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b <= $3))
+(14 rows)
+
+deallocate ab_q1;
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q1 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 4
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(6 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 2
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+   ->  Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < 3))
+(10 rows)
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q2 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+   ->  Seq Scan on ab_a2_b3 (never executed)
+         Filter: ((a >= $1) AND (a <= $2) AND (b < $0))
+(10 rows)
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+execute ab_q3 (1, 8);
+ a 
+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Result (actual rows=1 loops=1)
+   Subplans Pruned: 6
+   ->  Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+   ->  Seq Scan on ab_a3_b2 (never executed)
+         Filter: ((b >= $1) AND (b <= $2) AND (a < $0))
+(10 rows)
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q4 (1, 8);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((a >= $1) AND (a <= $2) AND (b < 4))
+(13 rows)
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q5 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 6
+                     ->  Parallel Seq Scan on ab_a1_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a1_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(13 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 3
+                     ->  Parallel Seq Scan on ab_a2_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a2_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b1 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b2 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+                     ->  Parallel Seq Scan on ab_a3_b3 (actual rows=0 loops=1)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(19 rows)
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Finalize Aggregate (actual rows=1 loops=1)
+   ->  Gather (actual rows=3 loops=1)
+         Workers Planned: 2
+         Workers Launched: 2
+         ->  Partial Aggregate (actual rows=1 loops=3)
+               ->  Parallel Append (actual rows=0 loops=3)
+                     Subplans Pruned: 8
+                     ->  Parallel Seq Scan on ab_a1_b1 (never executed)
+                           Filter: ((b < 4) AND (a = ANY (ARRAY[$1, $2, $3])))
+(9 rows)
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+analyze lprt_a;
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+execute ab_q6 (1, 2, 3);
+ avg 
+-----
+    
+(1 row)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{0,0,1}'::integer[]))
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+insert into lprt_a values(3),(3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=52 loops=2)
+                           Filter: (a = ANY ('{1,0,3}'::integer[]))
+                     ->  Append (actual rows=0 loops=104)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(36 rows)
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ 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=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=102)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (actual rows=0 loops=2)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+delete from lprt_a where a = 1;
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+                                           QUERY PLAN                                            
+-------------------------------------------------------------------------------------------------
+ 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=50 loops=2)
+                           Filter: (a = ANY ('{1,0,0}'::integer[]))
+                           Rows Removed by Filter: 1
+                     ->  Append (actual rows=0 loops=100)
+                           ->  Index Only Scan using ab_a1_b1_a_idx on ab_a1_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b2_a_idx on ab_a1_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a1_b3_a_idx on ab_a1_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b1_a_idx on ab_a2_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b2_a_idx on ab_a2_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a2_b3_a_idx on ab_a2_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b1_a_idx on ab_a3_b1 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b2_a_idx on ab_a3_b2 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+                           ->  Index Only Scan using ab_a3_b3_a_idx on ab_a3_b3 (never executed)
+                                 Index Cond: (a = a.a)
+                                 Heap Fetches: 0
+(37 rows)
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a (actual rows=102 loops=1)
+   InitPlan 2 (returns $1)
+     ->  Aggregate (actual rows=1 loops=1)
+           ->  Seq Scan on lprt_a lprt_a_1 (actual rows=102 loops=1)
+   ->  Bitmap Heap Scan on ab_a1_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a1_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a1_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b2 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b2_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a2_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a2_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b1 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b1_a_idx (never executed)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b2 (actual rows=0 loops=1)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b2_a_idx (actual rows=0 loops=1)
+               Index Cond: (a = $0)
+   ->  Bitmap Heap Scan on ab_a3_b3 (never executed)
+         Recheck Cond: (a = $0)
+         Filter: (b = $1)
+         ->  Bitmap Index Scan on ab_a3_b3_a_idx (never executed)
+               Index Cond: (a = $0)
+(52 rows)
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+drop table ab, lprt_a;
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=6 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=3 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 4
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=2 loops=1)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=2 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=2 loops=1)
+   ->  Append (actual rows=1 loops=2)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+(6 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+(2 rows)
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=23 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=5 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (actual rows=2 loops=5)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 10
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=3 loops=4)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 11
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=1 loops=2)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 < tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=3 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=5 loops=1)
+   ->  Append (actual rows=1 loops=5)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (actual rows=1 loops=2)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 2
+         ->  Index Only Scan using tprt3_idx on tprt_3 (actual rows=0 loops=3)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 1
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |   10
+  501 |   20
+  505 |   10
+  505 |   20
+  505 |  501
+  505 |  502
+ 1001 |   10
+ 1001 |   20
+ 1001 |  501
+ 1001 |  502
+ 1001 |  505
+ 1010 |   10
+ 1010 |   20
+ 1010 |  501
+ 1010 |  502
+ 1010 |  505
+ 1010 | 1001
+ 1011 |   10
+ 1011 |   20
+ 1011 |  501
+ 1011 |  502
+ 1011 |  505
+ 1011 | 1001
+(23 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+  501 |  501
+  505 |  505
+ 1001 | 1001
+(3 rows)
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+                                  QUERY PLAN                                   
+-------------------------------------------------------------------------------
+ Nested Loop (actual rows=1 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=1 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (actual rows=1 loops=1)
+               Index Cond: (col1 > tbl1.col1)
+               Heap Fetches: 1
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+ 4400 | 4500
+(1 row)
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+                               QUERY PLAN                               
+------------------------------------------------------------------------
+ Nested Loop (actual rows=0 loops=1)
+   ->  Seq Scan on tbl1 (actual rows=1 loops=1)
+   ->  Append (actual rows=0 loops=1)
+         ->  Index Only Scan using tprt1_idx on tprt_1 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt2_idx on tprt_2 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt3_idx on tprt_3 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt4_idx on tprt_4 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt5_idx on tprt_5 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+         ->  Index Only Scan using tprt6_idx on tprt_6 (never executed)
+               Index Cond: (col1 = tbl1.col1)
+               Heap Fetches: 0
+(21 rows)
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+ col1 | col1 
+------+------
+(0 rows)
+
+drop table tbl1, tprt;
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+execute part_abc_q1 (1, 2, 3);
+ a | b | c 
+---+---+---
+(0 rows)
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+                      QUERY PLAN                       
+-------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   ->  Seq Scan on part_abc_p1 (actual rows=0 loops=1)
+         Filter: ((a = $1) AND (b = $2) AND (c = $3))
+(3 rows)
+
+deallocate part_abc_q1;
+drop table part_abc;
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+ a | b 
+---+---
+(0 rows)
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2);
+ a | b 
+---+---
+(0 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_2_1 (actual rows=0 loops=1)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+                  QUERY PLAN                  
+----------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: (b = ANY (ARRAY[$1, $2]))
+(4 rows)
+
+deallocate q1;
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+execute q1 (1,2,3,4);
+ a | b 
+---+---
+(0 rows)
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (actual rows=0 loops=1)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+                               QUERY PLAN                                
+-------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Pruned: 1
+   ->  Seq Scan on listp_1_1 (never executed)
+         Filter: ((b = ANY (ARRAY[$1, $2])) AND ($3 <> b) AND ($4 <> b))
+(4 rows)
+
+drop table listp;
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: value
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (never executed)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (actual rows=0 loops=1)
+         Filter: (a = $0)
+(9 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   InitPlan 1 (returns $0)
+     ->  Seq Scan on boolvalues (actual rows=1 loops=1)
+           Filter: (NOT value)
+           Rows Removed by Filter: 1
+   ->  Seq Scan on boolp_f (actual rows=0 loops=1)
+         Filter: (a = $0)
+   ->  Seq Scan on boolp_t (never executed)
+         Filter: (a = $0)
+(9 rows)
+
+drop table boolp;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index d5ca3cb702..1d712b4ada 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -237,3 +237,347 @@ create table rparted_by_int2_maxvalue partition of rparted_by_int2 for values fr
 explain (costs off) select * from rparted_by_int2 where a > 100000000000000;
 
 drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, rp, coll_pruning_multi, like_op_noprune, lparted_by_int2, rparted_by_int2;
+
+
+--
+-- Test runtime partition pruning
+--
+create table ab (a int not null, b int not null) partition by list (a);
+create table ab_a2 partition of ab for values in(2) partition by list (b);
+create table ab_a2_b1 partition of ab_a2 for values in (1);
+create table ab_a2_b2 partition of ab_a2 for values in (2);
+create table ab_a2_b3 partition of ab_a2 for values in (3);
+create table ab_a1 partition of ab for values in(1) partition by list (b);
+create table ab_a1_b1 partition of ab_a1 for values in (1);
+create table ab_a1_b2 partition of ab_a1 for values in (2);
+create table ab_a1_b3 partition of ab_a1 for values in (3);
+create table ab_a3 partition of ab for values in(3) partition by list (b);
+create table ab_a3_b1 partition of ab_a3 for values in (1);
+create table ab_a3_b2 partition of ab_a3 for values in (2);
+create table ab_a3_b3 partition of ab_a3 for values in (3);
+
+prepare ab_q1 (int, int, int) as
+select * from ab where a between $1 and $2 and b <= $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+execute ab_q1 (1, 8, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (1, 2, 3);
+
+deallocate ab_q1;
+
+-- Runtime pruning after optimizer pruning
+prepare ab_q1 (int, int) as
+select a from ab where a between $1 and $2 and b < 3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+execute ab_q1 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 2);
+explain (analyze, costs off, summary off, timing off) execute ab_q1 (2, 4);
+
+-- Ensure a mix of external and exec params work together at different
+-- levels of partitioning.
+prepare ab_q2 (int, int) as
+select a from ab where a between $1 and $2 and b < (select 3);
+
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+execute ab_q2 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q2 (2, 2);
+
+-- As above, but with swap the exec param to the first partition level
+prepare ab_q3 (int, int) as
+select a from ab where b between $1 and $2 and a < (select 3);
+
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+execute ab_q3 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q3 (2, 2);
+
+-- Parallel append
+prepare ab_q4 (int, int) as
+select avg(a) from ab where a between $1 and $2 and b < 4;
+
+-- Encourage use of parallel plans
+set parallel_setup_cost = 0;
+set parallel_tuple_cost = 0;
+set min_parallel_table_scan_size = 0;
+set max_parallel_workers_per_gather = 2;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+execute ab_q4 (1, 8);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q4 (2, 2);
+
+-- Test run-time pruning with IN lists.
+prepare ab_q5 (int, int, int) as
+select avg(a) from ab where a in($1,$2,$3) and b < 4;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+execute ab_q5 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (1, 1, 1);
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (2, 3, 3);
+
+-- Try some params whose values do not belong to any partition.
+-- We'll still get a single subplan in this case, but it should not be scanned.
+explain (analyze, costs off, summary off, timing off) execute ab_q5 (33, 44, 55);
+
+-- Test parallel Append with IN list and parameterized nested loops
+create table lprt_a (a int not null);
+-- Insert some values we won't find in ab
+insert into lprt_a select 0 from generate_series(1,100);
+
+-- and insert some values that we should find.
+insert into lprt_a values(1),(1);
+
+analyze lprt_a;
+
+create index ab_a2_b1_a_idx on ab_a2_b1 (a);
+create index ab_a2_b2_a_idx on ab_a2_b2 (a);
+create index ab_a2_b3_a_idx on ab_a2_b3 (a);
+create index ab_a1_b1_a_idx on ab_a1_b1 (a);
+create index ab_a1_b2_a_idx on ab_a1_b2 (a);
+create index ab_a1_b3_a_idx on ab_a1_b3 (a);
+create index ab_a3_b1_a_idx on ab_a3_b1 (a);
+create index ab_a3_b2_a_idx on ab_a3_b2 (a);
+create index ab_a3_b3_a_idx on ab_a3_b3 (a);
+
+set enable_hashjoin = 0;
+set enable_mergejoin = 0;
+
+prepare ab_q6 (int, int, int) as
+select avg(ab.a) from ab inner join lprt_a a on ab.a = a.a where a.a in($1,$2,$3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+execute ab_q6 (1, 2, 3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (0, 0, 1);
+
+insert into lprt_a values(3),(3);
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 3);
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+delete from lprt_a where a = 1;
+
+explain (analyze, costs off, summary off, timing off) execute ab_q6 (1, 0, 0);
+
+reset enable_hashjoin;
+reset enable_mergejoin;
+reset parallel_setup_cost;
+reset parallel_tuple_cost;
+reset min_parallel_table_scan_size;
+reset max_parallel_workers_per_gather;
+
+-- Test run-time partition pruning with an initplan
+explain (analyze, costs off, summary off, timing off)
+select * from ab where a = (select max(a) from lprt_a) and b = (select max(a)-1 from lprt_a);
+
+deallocate ab_q1;
+deallocate ab_q2;
+deallocate ab_q3;
+deallocate ab_q4;
+deallocate ab_q5;
+deallocate ab_q6;
+
+drop table ab, lprt_a;
+
+-- Join
+create table tbl1(col1 int);
+insert into tbl1 values (501), (505);
+
+-- Basic table
+create table tprt (col1 int) partition by range (col1);
+create table tprt_1 partition of tprt for values from (1) to (501);
+create table tprt_2 partition of tprt for values from (501) to (1001);
+create table tprt_3 partition of tprt for values from (1001) to (2001);
+create table tprt_4 partition of tprt for values from (2001) to (3001);
+create table tprt_5 partition of tprt for values from (3001) to (4001);
+create table tprt_6 partition of tprt for values from (4001) to (5001);
+
+create index tprt1_idx on tprt_1 (col1);
+create index tprt2_idx on tprt_2 (col1);
+create index tprt3_idx on tprt_3 (col1);
+create index tprt4_idx on tprt_4 (col1);
+create index tprt5_idx on tprt_5 (col1);
+create index tprt6_idx on tprt_6 (col1);
+
+insert into tprt values (10), (20), (501), (502), (505), (1001), (4500);
+
+set enable_hashjoin = off;
+set enable_mergejoin = off;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Multiple partitions
+insert into tbl1 values (1001), (1010), (1011);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 > tprt.col1;
+
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 inner join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 > tprt.col1
+order by tbl1.col1, tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- Last partition
+delete from tbl1;
+insert into tbl1 values (4400);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 < tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 < tprt.col1
+order by tbl1.col1, tprt.col1;
+
+-- No matching partition
+delete from tbl1;
+insert into tbl1 values (10000);
+explain (analyze, costs off, summary off, timing off)
+select * from tbl1 join tprt on tbl1.col1 = tprt.col1;
+
+select tbl1.col1, tprt.col1 from tbl1
+inner join tprt on tbl1.col1 = tprt.col1
+order by tbl1.col1, tprt.col1;
+
+drop table tbl1, tprt;
+
+-- Test with columns defined in varying orders between each level
+create table part_abc (a int not null, b int not null, c int not null) partition by list (a);
+create table part_bac (b int not null, a int not null, c int not null) partition by list (b);
+create table part_cab (c int not null, a int not null, b int not null) partition by list (c);
+create table part_abc_p1 (a int not null, b int not null, c int not null);
+
+alter table part_abc attach partition part_bac for values in(1);
+alter table part_bac attach partition part_cab for values in(2);
+alter table part_cab attach partition part_abc_p1 for values in(3);
+
+prepare part_abc_q1 (int, int, int) as
+select * from part_abc where a = $1 and b = $2 and c = $3;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+execute part_abc_q1 (1, 2, 3);
+
+-- Single partition should be scanned.
+explain (analyze, costs off, summary off, timing off) execute part_abc_q1 (1, 2, 3);
+
+deallocate part_abc_q1;
+
+drop table part_abc;
+
+-- Ensure that an Append node properly handles a sub-partitioned table
+-- matching without any of its leaf partitions matching the clause.
+create table listp (a int, b int) partition by list (a);
+create table listp_1 partition of listp for values in(1) partition by list (b);
+create table listp_1_1 partition of listp_1 for values in(1);
+create table listp_2 partition of listp for values in(2) partition by list (b);
+create table listp_2_1 partition of listp_2 for values in(2);
+select * from listp where b = 1;
+
+-- Ensure that an Append node properly can handle selection of all first level
+-- partitions before finally detecting the correct set of 2nd level partitions
+-- which match the given parameter.
+prepare q1 (int,int) as select * from listp where b in ($1,$2);
+
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+execute q1 (1,2);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,1);
+
+explain (analyze, costs off, summary off, timing off)  execute q1 (2,2);
+
+-- Try with no matching partitions. One subplan should remain in this case,
+-- but it shouldn't be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (0,0);
+
+deallocate q1;
+
+-- Test more complex cases where a not-equal condition further eliminates partitions.
+prepare q1 (int,int,int,int) as select * from listp where b in($1,$2) and $3 <> b and $4 <> b;
+
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+execute q1 (1,2,3,4);
+
+-- Both partitions allowed by IN clause, but one disallowed by <> clause
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,0);
+
+-- Both partitions allowed by IN clause, then both excluded again by <> clauses.
+-- One subplan will remain in this case, but it should not be executed.
+explain (analyze, costs off, summary off, timing off)  execute q1 (1,2,2,1);
+
+drop table listp;
+
+-- Ensure runtime pruning works with initplans params with boolean types
+create table boolvalues (value bool not null);
+insert into boolvalues values('t'),('f');
+
+create table boolp (a bool) partition by list (a);
+create table boolp_t partition of boolp for values in('t');
+create table boolp_f partition of boolp for values in('f');
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where value);
+
+explain (analyze, costs off, summary off, timing off)
+select * from boolp where a = (select value from boolvalues where not value);
+
+drop table boolp;
-- 
2.16.2.windows.1

v23-0004-Allow-unneeded-MergeAppend-s-subnodes-to-be-prun.patchapplication/octet-stream; name=v23-0004-Allow-unneeded-MergeAppend-s-subnodes-to-be-prun.patchDownload
From 1b7a3b0d32b1dbc977253b34f5122ea797cd4b4d Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Sat, 7 Apr 2018 20:25:42 +1200
Subject: [PATCH v23 4/4] Allow unneeded MergeAppend's subnodes to be pruned at
 execution

Already supported for Append nodes, this commit allows partition pruning to
occur in MergeAppend using values which are only known during execution.
---
 src/backend/executor/nodeMergeAppend.c        | 136 ++++++++++++++++++++----
 src/backend/nodes/copyfuncs.c                 |   1 +
 src/backend/nodes/outfuncs.c                  |   2 +
 src/backend/nodes/readfuncs.c                 |   1 +
 src/backend/optimizer/plan/createplan.c       |  34 ++++++
 src/include/nodes/execnodes.h                 |   9 ++
 src/include/nodes/plannodes.h                 |   5 +
 src/test/regress/expected/partition_prune.out | 145 ++++++++++++++++++++++++++
 src/test/regress/sql/partition_prune.sql      |  41 ++++++++
 9 files changed, 353 insertions(+), 21 deletions(-)

diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..245c861764 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -39,6 +39,7 @@
 #include "postgres.h"
 
 #include "executor/execdebug.h"
+#include "executor/execPartition.h"
 #include "executor/nodeMergeAppend.h"
 #include "lib/binaryheap.h"
 #include "miscadmin.h"
@@ -65,8 +66,10 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 {
 	MergeAppendState *mergestate = makeNode(MergeAppendState);
 	PlanState **mergeplanstates;
+	Bitmapset  *validsubplans;
 	int			nplans;
-	int			i;
+	int			i,
+				j;
 	ListCell   *lc;
 
 	/* check for unsupported flags */
@@ -78,19 +81,89 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 */
 	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
 
-	/*
-	 * Set up empty vector of subplan states
-	 */
-	nplans = list_length(node->mergeplans);
-
-	mergeplanstates = (PlanState **) palloc0(nplans * sizeof(PlanState *));
-
 	/*
 	 * create new MergeAppendState for our node
 	 */
 	mergestate->ps.plan = (Plan *) node;
 	mergestate->ps.state = estate;
 	mergestate->ps.ExecProcNode = ExecMergeAppend;
+	mergestate->ms_noopscan = false;
+
+	/* If run-time partition pruning is enabled, then setup that up now */
+	if (node->part_prune_infos != NIL)
+	{
+		PartitionPruneState *prunestate;
+
+		ExecAssignExprContext(estate, &mergestate->ps);
+
+		prunestate = ExecSetupPartitionPruneState(&mergestate->ps,
+												  node->part_prune_infos);
+
+		/*
+		 * When there are external params matching the partition key we may be
+		 * able to prune away MergeAppend subplans now.
+		 */
+		if (!bms_is_empty(prunestate->extparams))
+		{
+			/* Determine which subplans match the external params */
+			validsubplans = ExecFindInitialMatchingSubPlans(prunestate,
+															list_length(node->mergeplans));
+
+			/*
+			 * If no subplans match the given parameters then we must handle
+			 * this case in a special way.  The problem here is that code in
+			 * explain.c requires a MergeAppend to have at least one subplan
+			 * in order for it to properly determine the Vars in that
+			 * subplan's targetlist.  We sidestep this issue by just
+			 * initializing the first subplan, but we set a noop flag so that
+			 * we never actually bother scanning it.
+			 */
+			if (bms_is_empty(validsubplans))
+			{
+				mergestate->ms_noopscan = true;
+
+				/* Mark the first as valid so that it's initialized below */
+				validsubplans = bms_make_singleton(0);
+			}
+
+			nplans = bms_num_members(validsubplans);
+		}
+		else
+		{
+			/* We'll need to initialize all subplans */
+			nplans = list_length(node->mergeplans);
+			validsubplans = bms_add_range(NULL, 0, nplans - 1);
+		}
+
+		/*
+		 * If there are no exec params then no further pruning can be done, we
+		 * can just set the valid subplans to all remaining subplans.
+		 * Otherwise we set the valid subplans to NULL so that they can be
+		 * determined during actual execution.
+		 */
+		if (bms_is_empty(prunestate->execparams))
+			mergestate->ms_valid_subplans = bms_add_range(NULL, 0, nplans - 1);
+		else
+			mergestate->ms_valid_subplans = NULL;
+
+
+		mergestate->ms_prune_state = prunestate;
+
+	}
+	else
+	{
+		nplans = list_length(node->mergeplans);
+
+		/*
+		 * When run-time partition pruning is not enabled we can just mark all
+		 * subplans as valid.
+		 */
+		mergestate->ms_valid_subplans = validsubplans =
+			bms_add_range(NULL, 0, nplans - 1);
+		mergestate->ms_prune_state = NULL;
+	}
+
+	mergeplanstates = (PlanState **) palloc(nplans * sizeof(PlanState *));
 	mergestate->mergeplans = mergeplanstates;
 	mergestate->ms_nplans = nplans;
 
@@ -101,26 +174,24 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	/*
 	 * Miscellaneous initialization
 	 *
-	 * MergeAppend plans don't have expression contexts because they never
-	 * call ExecQual or ExecProject.
-	 */
-
-	/*
 	 * MergeAppend nodes do have Result slots, which hold pointers to tuples,
 	 * so we have to initialize them.
 	 */
 	ExecInitResultTupleSlotTL(estate, &mergestate->ps);
 
 	/*
-	 * call ExecInitNode on each of the plans to be executed and save the
-	 * results into the array "mergeplans".
+	 * call ExecInitNode on each of the valid plans to be executed and save
+	 * the results into the mergeplanstates array.
 	 */
-	i = 0;
+	j = i = 0;
 	foreach(lc, node->mergeplans)
 	{
-		Plan	   *initNode = (Plan *) lfirst(lc);
+		if (bms_is_member(i, validsubplans))
+		{
+			Plan	   *initNode = (Plan *) lfirst(lc);
 
-		mergeplanstates[i] = ExecInitNode(initNode, estate, eflags);
+			mergeplanstates[j++] = ExecInitNode(initNode, estate, eflags);
+		}
 		i++;
 	}
 
@@ -178,11 +249,21 @@ ExecMergeAppend(PlanState *pstate)
 
 	if (!node->ms_initialized)
 	{
+		/* Handle the case for when all subplans were pruned */
+		if (node->ms_noopscan)
+			return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+		/* Determine minimum set of matching partitions, if not already set */
+		if (node->ms_valid_subplans == NULL)
+			node->ms_valid_subplans =
+				ExecFindMatchingSubPlans(node->ms_prune_state);
+
 		/*
-		 * First time through: pull the first tuple from each subplan, and set
-		 * up the heap.
+		 * First time through: pull the first tuple from each valid subplan,
+		 * and set up the heap.
 		 */
-		for (i = 0; i < node->ms_nplans; i++)
+		i = -1;
+		while ((i = bms_next_member(node->ms_valid_subplans, i)) >= 0)
 		{
 			node->ms_slots[i] = ExecProcNode(node->mergeplans[i]);
 			if (!TupIsNull(node->ms_slots[i]))
@@ -295,6 +376,19 @@ ExecReScanMergeAppend(MergeAppendState *node)
 {
 	int			i;
 
+	/*
+	 * If any of the parameters being used for partition pruning have changed,
+	 * then we'd better unset the valid subplans so that they are reselected
+	 * for the new parameter values.
+	 */
+	if (node->ms_prune_state &&
+		bms_overlap(node->ps.chgParam,
+					node->ms_prune_state->execparams))
+	{
+		bms_free(node->ms_valid_subplans);
+		node->ms_valid_subplans = NULL;
+	}
+
 	for (i = 0; i < node->ms_nplans; i++)
 	{
 		PlanState  *subnode = node->mergeplans[i];
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 765f290bc7..0c2041bd2e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -276,6 +276,7 @@ _copyMergeAppend(const MergeAppend *from)
 	COPY_POINTER_FIELD(sortOperators, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(collations, from->numCols * sizeof(Oid));
 	COPY_POINTER_FIELD(nullsFirst, from->numCols * sizeof(bool));
+	COPY_NODE_FIELD(part_prune_infos);
 
 	return newnode;
 }
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index c274adaacc..14567048be 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -451,6 +451,8 @@ _outMergeAppend(StringInfo str, const MergeAppend *node)
 	appendStringInfoString(str, " :nullsFirst");
 	for (i = 0; i < node->numCols; i++)
 		appendStringInfo(str, " %s", booltostr(node->nullsFirst[i]));
+
+	WRITE_NODE_FIELD(part_prune_infos);
 }
 
 static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 93785e3bdf..2e9da37fc3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1714,6 +1714,7 @@ _readMergeAppend(void)
 	READ_OID_ARRAY(sortOperators, local_node->numCols);
 	READ_OID_ARRAY(collations, local_node->numCols);
 	READ_BOOL_ARRAY(nullsFirst, local_node->numCols);
+	READ_NODE_FIELD(part_prune_infos);
 
 	READ_DONE();
 }
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d515b1ea78..aac05917ee 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -1144,6 +1144,8 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 	List	   *pathkeys = best_path->path.pathkeys;
 	List	   *subplans = NIL;
 	ListCell   *subpaths;
+	RelOptInfo *rel = best_path->path.parent;
+	List	   *partpruneinfos = NIL;
 
 	/*
 	 * We don't have the actual creation of the MergeAppend node split out
@@ -1229,8 +1231,40 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
 		subplans = lappend(subplans, subplan);
 	}
 
+	if (rel->reloptkind == RELOPT_BASEREL &&
+		best_path->partitioned_rels != NIL)
+	{
+		List	   *prunequal;
+
+		prunequal = extract_actual_clauses(rel->baserestrictinfo, false);
+
+		if (best_path->path.param_info)
+		{
+
+			List	   *prmquals = best_path->path.param_info->ppi_clauses;
+
+			prmquals = extract_actual_clauses(prmquals, false);
+			prmquals = (List *) replace_nestloop_params(root,
+														(Node *) prmquals);
+
+			prunequal = list_concat(prunequal, prmquals);
+		}
+
+		/*
+		 * If any quals exist, then these may be useful to allow us to perform
+		 * further partition pruning during execution.  We'll generate a
+		 * PartitionPruneInfo for each partitioned rel to store these quals
+		 * and allow translation of partition indexes into subpath indexes.
+		 */
+		if (prunequal != NIL)
+			partpruneinfos = make_partition_pruneinfo(root,
+													  best_path->partitioned_rels,
+													  best_path->subpaths, prunequal);
+	}
+
 	node->partitioned_rels = best_path->partitioned_rels;
 	node->mergeplans = subplans;
+	node->part_prune_infos = partpruneinfos;
 
 	return (Plan *) node;
 }
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index efd0752389..594ccf47ec 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1158,6 +1158,12 @@ struct AppendState
  *		slots			current output tuple of each subplan
  *		heap			heap of active tuples
  *		initialized		true if we have fetched first tuple from each subplan
+ *		noopscan		true if partition pruning proved that none of the
+ *						mergeplans can contain a record to satisfy this query.
+ *		prune_state		details required to allow partitions to be
+ *						eliminated from the scan, or NULL if not possible.
+ *		valid_subplans	for runtime pruning, valid mergeplans indexes to
+ *						scan.
  * ----------------
  */
 typedef struct MergeAppendState
@@ -1170,6 +1176,9 @@ typedef struct MergeAppendState
 	TupleTableSlot **ms_slots;	/* array of length ms_nplans */
 	struct binaryheap *ms_heap; /* binary heap of slot indices */
 	bool		ms_initialized; /* are subplans started? */
+	bool		ms_noopscan;	/* true if no subplans need scanned */
+	struct PartitionPruneState *ms_prune_state;
+	Bitmapset  *ms_valid_subplans;
 } MergeAppendState;
 
 /* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index c3e5c2c79f..a7dbd31466 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -280,6 +280,11 @@ typedef struct MergeAppend
 	Oid		   *sortOperators;	/* OIDs of operators to sort them by */
 	Oid		   *collations;		/* OIDs of collations */
 	bool	   *nullsFirst;		/* NULLS FIRST/LAST directions */
+
+	/*
+	 * Mapping details for run-time subplan pruning, one per partitioned_rels
+	 */
+	List	   *part_prune_infos;
 } MergeAppend;
 
 /* ----------------
diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out
index 99d94d9a9d..9677f73205 100644
--- a/src/test/regress/expected/partition_prune.out
+++ b/src/test/regress/expected/partition_prune.out
@@ -2466,3 +2466,148 @@ select * from boolp where a = (select value from boolvalues where not value);
 (9 rows)
 
 drop table boolp;
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+execute mt_q1(0);
+ a  
+----
+  5
+ 15
+ 25
+(3 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=2 loops=1)
+   Sort Key: ma_test_p2.a
+   Subplans Pruned: 1
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 9
+         Heap Fetches: 10
+(13 rows)
+
+execute mt_q1(15);
+ a  
+----
+ 15
+ 25
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+                                     QUERY PLAN                                     
+------------------------------------------------------------------------------------
+ Merge Append (actual rows=1 loops=1)
+   Sort Key: ma_test_p3.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Rows Removed by Filter: 4
+         Heap Fetches: 5
+(8 rows)
+
+execute mt_q1(25);
+ a  
+----
+ 25
+(1 row)
+
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+                                 QUERY PLAN                                  
+-----------------------------------------------------------------------------
+ Merge Append (actual rows=0 loops=1)
+   Sort Key: ma_test_p1.a
+   Subplans Pruned: 2
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Filter: ((a % 10) = 5)
+         Heap Fetches: 0
+(7 rows)
+
+execute mt_q1(35);
+ a 
+---
+(0 rows)
+
+deallocate mt_q1;
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+                                                   QUERY PLAN                                                    
+-----------------------------------------------------------------------------------------------------------------
+ Merge Append (actual rows=20 loops=1)
+   Sort Key: ma_test_p1.a
+   InitPlan 2 (returns $1)
+     ->  Result (actual rows=1 loops=1)
+           InitPlan 1 (returns $0)
+             ->  Limit (actual rows=1 loops=1)
+                   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
+                         Index Cond: (a IS NOT NULL)
+                         Heap Fetches: 1
+   ->  Index Only Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
+         Index Cond: (a >= $1)
+         Heap Fetches: 0
+   ->  Index Only Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+   ->  Index Only Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
+         Index Cond: (a >= $1)
+         Heap Fetches: 10
+(18 rows)
+
+reset enable_seqscan;
+reset enable_sort;
+drop table ma_test;
diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql
index 1d712b4ada..688db55aec 100644
--- a/src/test/regress/sql/partition_prune.sql
+++ b/src/test/regress/sql/partition_prune.sql
@@ -581,3 +581,44 @@ explain (analyze, costs off, summary off, timing off)
 select * from boolp where a = (select value from boolvalues where not value);
 
 drop table boolp;
+
+--
+-- Test run-time pruning of MergeAppend subnodes
+--
+set enable_seqscan = off;
+set enable_sort = off;
+create table ma_test (a int) partition by range (a);
+create table ma_test_p1 partition of ma_test for values from (0) to (10);
+create table ma_test_p2 partition of ma_test for values from (10) to (20);
+create table ma_test_p3 partition of ma_test for values from (20) to (30);
+insert into ma_test select x from generate_series(0,29) t(x);
+create index on ma_test (a);
+
+analyze ma_test;
+prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
+
+-- Execute query 5 times to allow choose_custom_plan
+-- to start considering a generic plan.
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+execute mt_q1(0);
+
+explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
+execute mt_q1(15);
+explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
+execute mt_q1(25);
+-- Ensure MergeAppend behaves correctly when no subplans match
+explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
+execute mt_q1(35);
+
+deallocate mt_q1;
+
+-- ensure initplan params properly prune partitions
+explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
+
+reset enable_seqscan;
+reset enable_sort;
+
+drop table ma_test;
-- 
2.16.2.windows.1

#132Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#130)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Amit Langote wrote:

See if the attached makes it any better.

Now I know we don't have the runtime pruning in yet, but since the
proposed patch would extend its functionality I have included its
description in the comment.

Thanks!

I edited it as attached, to 1. avoid mentioning functionality that
doesn't yet exist, and 2. avoid excessive internal detail (we want a
high-level overview here), which from experience gets outdated pretty
quickly.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

expand-partprune-header-comment-2.patchtext/plain; charset=us-asciiDownload
commit 28768290d8d9c58e76e2594a14d5b1f8465ba262
Author:     Alvaro Herrera <alvherre@alvh.no-ip.org>
AuthorDate: Sat Apr 7 08:44:12 2018 -0300
CommitDate: Sat Apr 7 09:20:58 2018 -0300

    Add some documentation to partprune.c
    
    Author: Amit Langote
    Discussion: https://postgr.es/m/CA+HiwqGzq4D6z=8R0AP+XhbTFCQ-4Ct+t2ekqjE9Fpm84_JUGg@mail.gmail.com

diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 959ee1643d..07b8057e3f 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -1,10 +1,27 @@
 /*-------------------------------------------------------------------------
  *
  * partprune.c
- *		Parses clauses attempting to match them up to partition keys of a
- *		given relation and generates a set of "pruning steps", which can be
- *		later "executed" either from the planner or the executor to determine
- *		the minimum set of partitions which match the given clauses.
+ *		Support for partition pruning during query planning
+ *
+ * This module implements partition pruning using the information contained in
+ * table's partition descriptor and query clauses.
+ *
+ * During planning, clauses that can be matched to the table's partition key
+ * are turned into a set of "pruning steps", which are then executed to
+ * produce a set of RTIs of partitions whose bounds satisfy the constraints in
+ * the step.  Partitions not in the set are said to have been pruned.
+ *
+ * There are two kinds of pruning steps: a "base" pruning step, which contains
+ * information extracted from one or more clauses that are matched to the
+ * (possibly multi-column) partition key, such as the expressions whose values
+ * to match against partition bounds and operator strategy to associate to
+ * each expression.  The other kind is a "combine" pruning step, which combines
+ * the outputs of some other steps using the appropriate combination method.
+ * All steps that are constructed are executed in succession such that for any
+ * "combine" step, all of the steps whose output it depends on are executed
+ * first and their ouput preserved.
+ *
+ * See gen_partprune_steps_internal() for more details on step generation.
  *
  * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
#133David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#132)
Re: [HACKERS] Runtime Partition Pruning

On 8 April 2018 at 00:23, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I edited it as attached, to 1. avoid mentioning functionality that
doesn't yet exist, and 2. avoid excessive internal detail (we want a
high-level overview here), which from experience gets outdated pretty
quickly.

It's not exactly wrong but:

+ * are turned into a set of "pruning steps", which are then executed to
+ * produce a set of RTIs of partitions whose bounds satisfy the constraints in
+ * the step.  Partitions not in the set are said to have been pruned.

It's only prune_append_rel_partitions which is only used for the
planner's pruning needs that converts the partition indexes to RTIs.
Would it be better to mention that the output is partition indexes?
Maybe:

"which are then executed to produce a set of partition indexes whose
bounds satisfy the constraints in the step. These partition indexes
may then be translated into RTIs", or maybe even not mention the RTIs.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#134Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#133)
Re: [HACKERS] Runtime Partition Pruning

David Rowley wrote:

It's not exactly wrong but:

+ * are turned into a set of "pruning steps", which are then executed to
+ * produce a set of RTIs of partitions whose bounds satisfy the constraints in
+ * the step.  Partitions not in the set are said to have been pruned.

It's only prune_append_rel_partitions which is only used for the
planner's pruning needs that converts the partition indexes to RTIs.
Would it be better to mention that the output is partition indexes?
Maybe:

"which are then executed to produce a set of partition indexes whose
bounds satisfy the constraints in the step. These partition indexes
may then be translated into RTIs", or maybe even not mention the RTIs.

Amit had it as "indexes" also in his original. I wanted to avoid using
the "indexes" word alone, whose meaning is so overloaded. How about
this?

"... which are then executed to produce a set of partitions (as indexes
of resultRelInfo->part_rels array) that satisfy the constraints in the
step".

Maybe "the boundinfo array" instead of part_rels, which as I understand
also uses the same indexing as the other array, and partprune mostly
works based on boundinfo anyway?

Not mentioning RTIs seems fine.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#135David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#134)
Re: [HACKERS] Runtime Partition Pruning

On 8 April 2018 at 01:18, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Amit had it as "indexes" also in his original. I wanted to avoid using
the "indexes" word alone, whose meaning is so overloaded.

hmm, good point.

How about this?
"... which are then executed to produce a set of partitions (as indexes
of resultRelInfo->part_rels array) that satisfy the constraints in the
step".

Works for me, but with RelOptInfo rather than resultRelInfo.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#136Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#135)
Re: [HACKERS] Runtime Partition Pruning

David Rowley wrote:

On 8 April 2018 at 01:18, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Amit had it as "indexes" also in his original. I wanted to avoid using
the "indexes" word alone, whose meaning is so overloaded.

hmm, good point.

How about this?
"... which are then executed to produce a set of partitions (as indexes
of resultRelInfo->part_rels array) that satisfy the constraints in the
step".

Works for me, but with RelOptInfo rather than resultRelInfo.

Oops, sorry about that. Pushed now, adding one line to
get_matching_partitions also.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#137Jesper Pedersen
jesper.pedersen@redhat.com
In reply to: David Rowley (#131)
Re: [HACKERS] Runtime Partition Pruning

Hi,

On 04/07/2018 04:45 AM, David Rowley wrote:

Ok, so I've gone and done this.

PartitionPruning has become PartitionPruneState
PartitionRelPruning has become PartitionPruningData

I've changed pointers to PartitionPruneStates to be named prunestate,
sometimes having the node prefix; as_, ma_, in these cases prune and
state are separated with a _ which seems to be the general rule for
executor state struct members.

Generally, pointers to PartitionPruningData are now named pprune.
Hopefully, that's ok, as this was the name previously used for
PartitionPruning pointers.

I applied the patch to get rid of as_noop_scan in favour of using a
special as_whichplan value. There was already one special value
(INVALID_SUBPLAN_INDEX), so seemed better to build on that rather than
inventing something new. This also means we don't have to make the
AppendState struct and wider too, which seems like a good thing to try
to do.

I made all the fixups which I mentioned in my review earlier and also
re-removed the resultRelation parameter from make_partition_pruneinfo.
It sneaked back into v22.

v23 is attached.

Passes check-world.

Changing explain.c to "Subplans Removed" as suggested by you in [1]/messages/by-id/CAKJS1f99JnkbOshdV_4zoJZ96DPtKeHMHv43JRL_ZdHRkkVKCA@mail.gmail.com is a
good idea.

[1]: /messages/by-id/CAKJS1f99JnkbOshdV_4zoJZ96DPtKeHMHv43JRL_ZdHRkkVKCA@mail.gmail.com
/messages/by-id/CAKJS1f99JnkbOshdV_4zoJZ96DPtKeHMHv43JRL_ZdHRkkVKCA@mail.gmail.com

Best regards,
Jesper

#138Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#131)
Re: [HACKERS] Runtime Partition Pruning

I pushed this patch -- 0001, 0002 and 0003 only. I did not include
anything from 0004 and 0005; I didn't even get to the point of reading
them, so that I could focus on the first part.

I did not find anything to complain about. I made a few adjustments and
asked David to supply a paragraph for perform.sgml (the "Using EXPLAIN"
section) which is included here. I also adopted Jesper's (actually
David's) suggestion of changing "Partitions Pruned" to "Partitions
Removed" in the EXPLAIN output.

I had reservations about a relation_open() in the new executor code. It
seemed a bit odd; we don't have any other relation_open in the executor
anywhere. However, setting up the pruneinfo needs some stuff from
relcache that I don't see a reasonable mechanism to pass through
planner. I asked Andres about it on IM and while he didn't endorse the
patch in any way, his quick opinion was that "it wasn't entirely
insane". I verified that we already hold lock on the relation.

While we didn't get fast pruning support for MergeAppend or the
DELETE/UPDATE parts, I think those are valuable and recommend to
resubmit those for PG12.

Thank you!

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#139David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#138)
Re: [HACKERS] Runtime Partition Pruning

On 8 April 2018 at 09:13, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I pushed this patch -- 0001, 0002 and 0003 only. I did not include
anything from 0004 and 0005; I didn't even get to the point of reading
them, so that I could focus on the first part.

Oh great! Thank you for working on this and pushing it, especially so
during your weekend.

While we didn't get fast pruning support for MergeAppend or the
DELETE/UPDATE parts, I think those are valuable and recommend to
resubmit those for PG12.

Thanks. I'll certainly be doing that.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#140Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#138)
Re: [HACKERS] Runtime Partition Pruning

On Sat, Apr 7, 2018 at 5:13 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I had reservations about a relation_open() in the new executor code. It
seemed a bit odd; we don't have any other relation_open in the executor
anywhere. However, setting up the pruneinfo needs some stuff from
relcache that I don't see a reasonable mechanism to pass through
planner. I asked Andres about it on IM and while he didn't endorse the
patch in any way, his quick opinion was that "it wasn't entirely
insane". I verified that we already hold lock on the relation.

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#141Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Robert Haas (#140)
Re: [HACKERS] Runtime Partition Pruning

Robert Haas wrote:

On Sat, Apr 7, 2018 at 5:13 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I had reservations about a relation_open() in the new executor code. It
seemed a bit odd; we don't have any other relation_open in the executor
anywhere. However, setting up the pruneinfo needs some stuff from
relcache that I don't see a reasonable mechanism to pass through
planner. I asked Andres about it on IM and while he didn't endorse the
patch in any way, his quick opinion was that "it wasn't entirely
insane". I verified that we already hold lock on the relation.

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

Yeah.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#142Robert Haas
robertmhaas@gmail.com
In reply to: Alvaro Herrera (#141)
Re: [HACKERS] Runtime Partition Pruning

On Mon, Apr 9, 2018 at 2:28 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Robert Haas wrote:

On Sat, Apr 7, 2018 at 5:13 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I had reservations about a relation_open() in the new executor code. It
seemed a bit odd; we don't have any other relation_open in the executor
anywhere. However, setting up the pruneinfo needs some stuff from
relcache that I don't see a reasonable mechanism to pass through
planner. I asked Andres about it on IM and while he didn't endorse the
patch in any way, his quick opinion was that "it wasn't entirely
insane". I verified that we already hold lock on the relation.

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

Yeah.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

Well what I'm saying is that it shouldn't be necessary. If the
relations are being opened already and the pointers to the relcache
entries are being saved someplace, you shouldn't need to re-open them
elsewhere to get pointers to the relcache entries.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#143Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Robert Haas (#142)
Re: [HACKERS] Runtime Partition Pruning

Robert Haas wrote:

On Mon, Apr 9, 2018 at 2:28 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Robert Haas wrote:

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

Well what I'm saying is that it shouldn't be necessary. If the
relations are being opened already and the pointers to the relcache
entries are being saved someplace, you shouldn't need to re-open them
elsewhere to get pointers to the relcache entries.

Oh, okay. I can look into that.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#144Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Robert Haas (#142)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Robert Haas wrote:

On Mon, Apr 9, 2018 at 2:28 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

Well what I'm saying is that it shouldn't be necessary. If the
relations are being opened already and the pointers to the relcache
entries are being saved someplace, you shouldn't need to re-open them
elsewhere to get pointers to the relcache entries.

I looked a bit more into this. It turns out that we have indeed opened
the relation before -- first in parserOpenTable (for addRangeTableEntry),
then in expandRTE, then in QueryRewrite, then in subquery_planner, then
in get_relation_info.

So, frankly, since each module thinks it's okay to open it every once in
a while, I'm not sure we should be terribly stressed about doing it once
more for partition pruning. Particularly since communicating the
pointer seems to be quite troublesome.

To figure out, I used the attached patch (not intended for application)
to add a backtrace to each log message, plus a couple of accusatory
elog() calls in relation_open and ExecSetupPartitionPruneState.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

0001-errbacktrace.patchtext/plain; charset=us-asciiDownload
From 5d682c8cb31f79c79c2ba4cafeb876f10b072cfc Mon Sep 17 00:00:00 2001
From: Alvaro Herrera <alvherre@alvh.no-ip.org>
Date: Tue, 10 Apr 2018 18:30:29 -0300
Subject: [PATCH] errbacktrace

---
 src/backend/utils/error/elog.c      | 59 +++++++++++++++++++++++++++++++++++++
 src/include/postgres_ext.h          |  1 +
 src/include/utils/elog.h            |  3 ++
 src/interfaces/libpq/fe-protocol3.c |  3 ++
 4 files changed, 66 insertions(+)

diff --git a/src/backend/utils/error/elog.c b/src/backend/utils/error/elog.c
index 16531f7a0f..e531d0009e 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -62,6 +62,7 @@
 #ifdef HAVE_SYSLOG
 #include <syslog.h>
 #endif
+#include <execinfo.h>
 
 #include "access/transam.h"
 #include "access/xact.h"
@@ -493,6 +494,8 @@ errfinish(int dummy,...)
 		pfree(edata->hint);
 	if (edata->context)
 		pfree(edata->context);
+	if (edata->backtrace)
+		pfree(edata->backtrace);
 	if (edata->schema_name)
 		pfree(edata->schema_name);
 	if (edata->table_name)
@@ -811,6 +814,41 @@ errmsg(const char *fmt,...)
 	return 0;					/* return value does not matter */
 }
 
+#define BACKTRACE_FRAMES 100
+int
+errbacktrace(void)
+{
+	ErrorData   *edata = &errordata[errordata_stack_depth];
+	MemoryContext oldcontext;
+	void	   *buf[BACKTRACE_FRAMES];
+	int			nframes;
+	char	  **strfrms;
+	StringInfoData errtrace;
+	int			i;
+
+	recursion_depth++;
+	CHECK_STACK_DEPTH();
+	oldcontext = MemoryContextSwitchTo(edata->assoc_context);
+
+	nframes = backtrace(buf, BACKTRACE_FRAMES);
+	strfrms = backtrace_symbols(buf, nframes);
+	if (strfrms == NULL)
+		return 0;
+
+	initStringInfo(&errtrace);
+
+	/* the first frame is always errbacktrace itself, so skip it */
+	for (i = 1; i < nframes; i++)
+		appendStringInfo(&errtrace, "%s\n", strfrms[i]);
+	free(strfrms);
+
+	edata->backtrace = errtrace.data;
+
+	MemoryContextSwitchTo(oldcontext);
+	recursion_depth--;
+
+	return 0;
+}
 
 /*
  * errmsg_internal --- add a primary error message text to the current error
@@ -1522,6 +1560,8 @@ CopyErrorData(void)
 		newedata->hint = pstrdup(newedata->hint);
 	if (newedata->context)
 		newedata->context = pstrdup(newedata->context);
+	if (newedata->backtrace)
+		newedata->backtrace = pstrdup(newedata->backtrace);
 	if (newedata->schema_name)
 		newedata->schema_name = pstrdup(newedata->schema_name);
 	if (newedata->table_name)
@@ -1560,6 +1600,8 @@ FreeErrorData(ErrorData *edata)
 		pfree(edata->hint);
 	if (edata->context)
 		pfree(edata->context);
+	if (edata->backtrace)
+		pfree(edata->backtrace);
 	if (edata->schema_name)
 		pfree(edata->schema_name);
 	if (edata->table_name)
@@ -1635,6 +1677,8 @@ ThrowErrorData(ErrorData *edata)
 		newedata->hint = pstrdup(edata->hint);
 	if (edata->context)
 		newedata->context = pstrdup(edata->context);
+	if (edata->backtrace)
+		newedata->backtrace = pstrdup(newedata->backtrace);
 	/* assume message_id is not available */
 	if (edata->schema_name)
 		newedata->schema_name = pstrdup(edata->schema_name);
@@ -1702,6 +1746,8 @@ ReThrowError(ErrorData *edata)
 		newedata->hint = pstrdup(newedata->hint);
 	if (newedata->context)
 		newedata->context = pstrdup(newedata->context);
+	if (newedata->backtrace)
+		newedata->backtrace = pstrdup(newedata->backtrace);
 	if (newedata->schema_name)
 		newedata->schema_name = pstrdup(newedata->schema_name);
 	if (newedata->table_name)
@@ -2932,6 +2978,13 @@ send_message_to_server_log(ErrorData *edata)
 			append_with_tabs(&buf, edata->context);
 			appendStringInfoChar(&buf, '\n');
 		}
+		if (edata->backtrace)
+		{
+			log_line_prefix(&buf, edata);
+			appendStringInfoString(&buf, _("BACKTRACE:  "));
+			append_with_tabs(&buf, edata->backtrace);
+			appendStringInfoChar(&buf, '\n');
+		}
 		if (Log_error_verbosity >= PGERROR_VERBOSE)
 		{
 			/* assume no newlines in funcname or filename... */
@@ -3209,6 +3262,12 @@ send_message_to_frontend(ErrorData *edata)
 			err_sendstring(&msgbuf, edata->context);
 		}
 
+		if (edata->backtrace)
+		{
+			pq_sendbyte(&msgbuf, PG_DIAG_BACKTRACE);
+			err_sendstring(&msgbuf, edata->backtrace);
+		}
+
 		if (edata->schema_name)
 		{
 			pq_sendbyte(&msgbuf, PG_DIAG_SCHEMA_NAME);
diff --git a/src/include/postgres_ext.h b/src/include/postgres_ext.h
index fdb61b7cf5..799f0009f0 100644
--- a/src/include/postgres_ext.h
+++ b/src/include/postgres_ext.h
@@ -62,6 +62,7 @@ typedef PG_INT64_TYPE pg_int64;
 #define PG_DIAG_INTERNAL_POSITION 'p'
 #define PG_DIAG_INTERNAL_QUERY	'q'
 #define PG_DIAG_CONTEXT			'W'
+#define PG_DIAG_BACKTRACE		'B'
 #define PG_DIAG_SCHEMA_NAME		's'
 #define PG_DIAG_TABLE_NAME		't'
 #define PG_DIAG_COLUMN_NAME		'c'
diff --git a/src/include/utils/elog.h b/src/include/utils/elog.h
index 7a9ba7f2ff..a4343794b0 100644
--- a/src/include/utils/elog.h
+++ b/src/include/utils/elog.h
@@ -170,6 +170,8 @@ extern int	errcontext_msg(const char *fmt,...) pg_attribute_printf(1, 2);
 extern int	errhidestmt(bool hide_stmt);
 extern int	errhidecontext(bool hide_ctx);
 
+extern int	errbacktrace(void);
+
 extern int	errfunction(const char *funcname);
 extern int	errposition(int cursorpos);
 
@@ -345,6 +347,7 @@ typedef struct ErrorData
 	char	   *detail_log;		/* detail error message for server log only */
 	char	   *hint;			/* hint message */
 	char	   *context;		/* context message */
+	char	   *backtrace;		/* backtrace */
 	const char *message_id;		/* primary message's id (original string) */
 	char	   *schema_name;	/* name of schema */
 	char	   *table_name;		/* name of table */
diff --git a/src/interfaces/libpq/fe-protocol3.c b/src/interfaces/libpq/fe-protocol3.c
index d3ca5d25f6..3d9e40a09c 100644
--- a/src/interfaces/libpq/fe-protocol3.c
+++ b/src/interfaces/libpq/fe-protocol3.c
@@ -1079,6 +1079,9 @@ pqBuildErrorMessage3(PQExpBuffer msg, const PGresult *res,
 				appendPQExpBuffer(msg, libpq_gettext("CONTEXT:  %s\n"),
 								  val);
 		}
+		val = PQresultErrorField(res, PG_DIAG_BACKTRACE);
+		if (val)
+			appendPQExpBuffer(msg, libpq_gettext("BACKTRACE:  %s\n"), val);
 	}
 	if (verbosity == PQERRORS_VERBOSE)
 	{
-- 
2.11.0

#145Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Alvaro Herrera (#144)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/11 6:32, Alvaro Herrera wrote:

Robert Haas wrote:

On Mon, Apr 9, 2018 at 2:28 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

Well what I'm saying is that it shouldn't be necessary. If the
relations are being opened already and the pointers to the relcache
entries are being saved someplace, you shouldn't need to re-open them
elsewhere to get pointers to the relcache entries.

I looked a bit more into this. It turns out that we have indeed opened
the relation before -- first in parserOpenTable (for addRangeTableEntry),
then in expandRTE, then in QueryRewrite, then in subquery_planner, then
in get_relation_info.

So, frankly, since each module thinks it's okay to open it every once in
a while, I'm not sure we should be terribly stressed about doing it once
more for partition pruning. Particularly since communicating the
pointer seems to be quite troublesome.

Maybe, Robert was saying somewhere in the executor itself, before
ExecInitAppend, or more precisely, ExecSetupPartitionPruneState is called.
But that doesn't seem to be the case.

For the result relation case (insert/update/delete on a partitioned
table), we don't need to do extra relation_opens, as InitPlan opens the
needed relations when building the ResultRelInfo(s), from where they're
later accessed, for example, in ExecInitModifyTable.

However, in the Append/MergeAppend cases, we don't, at any earlier point
in the executor, open the partitioned tables. InitPlan doesn't touch
them. In ExecInitAppend, ExecLockNonLeafAppendTables that we call before
calling ExecSetupPartitionPruneState does not open, just locks them using
LockRelationOid.

So, relation_open on partitioned tables in ExecSetupPartitionPruneState
seem to be the first time they're opened *in the executor*.

Thanks,
Amit

#146David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#144)
Re: [HACKERS] Runtime Partition Pruning

On 11 April 2018 at 09:32, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Robert Haas wrote:

On Mon, Apr 9, 2018 at 2:28 PM, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

I don't get this. The executor surely had to (and did) open all of
the relations somewhere even before this patch.

I was worried that this coding could be seen as breaking modularity, or
trying to do excessive work. However, after looking closer at it, it
doesn't really look like it's the case. So, nevermind.

Well what I'm saying is that it shouldn't be necessary. If the
relations are being opened already and the pointers to the relcache
entries are being saved someplace, you shouldn't need to re-open them
elsewhere to get pointers to the relcache entries.

I looked a bit more into this. It turns out that we have indeed opened
the relation before -- first in parserOpenTable (for addRangeTableEntry),
then in expandRTE, then in QueryRewrite, then in subquery_planner, then
in get_relation_info.

So, frankly, since each module thinks it's okay to open it every once in
a while, I'm not sure we should be terribly stressed about doing it once
more for partition pruning. Particularly since communicating the
pointer seems to be quite troublesome.

I guess the problem there would be there's nothing to say that parse
analysis will shortly be followed by a call to the planner, and a call
to the planner does not mean the plan is about to be executed. So I
don't think it would be possible to keep pointers to relcache entries
between these modules, and it would be hard to determine whose
responsibility it would be to call relation_close().

It might be possible to do something better in each module by keeping
an array indexed by RTI which have each entry NULL initially then on
first relation_open set the element in the array to that pointer.

This might mean we'd save a few relation_open calls, but I don't know
if there would be a way to somehow remove the Relation from the array
on relation_close. Having something like this might mean we could
detect lock upgrade hazards more easily, but the whole thing is a
cache on top of a cache which does seem a bit weird. relation_open()
should be pretty cheap if the relation is already open. It's just a
hash table lookup. What is described above just changes that to an
array lookup. It also does nothing for index_open.

However, something like the above would simplify
ExecLockNonLeafAppendTables() a bit and get rid of the O(N^2) which
checks the partition is not a result relation.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#147Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#146)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Apr 12, 2018 at 12:40 AM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

I guess the problem there would be there's nothing to say that parse
analysis will shortly be followed by a call to the planner, and a call
to the planner does not mean the plan is about to be executed. So I
don't think it would be possible to keep pointers to relcache entries
between these modules, and it would be hard to determine whose
responsibility it would be to call relation_close().

Yeah, that's definitely a non-starter.

It might be possible to do something better in each module by keeping
an array indexed by RTI which have each entry NULL initially then on
first relation_open set the element in the array to that pointer.

I'm not sure that makes a lot of sense in the planner, but in the
executor it might be a good idea. See also
/messages/by-id/CA+TgmoYKToP4-adCFFRNrO21OGuH=phx-fiB1dYoqksNYX6YHQ@mail.gmail.com
for related discussion. I think that a coding pattern where we rely
on relation_open(..., NoLock) is inherently dangerous -- it's too easy
to be wrong about whether the lock is sure to have been taken. It
would be much better to open the relation once and hold onto the
pointer, not just for performance reasons, but for robustness.

BTW, looking at ExecSetupPartitionPruneState:

/*
* Create a sub memory context which we'll use when making calls to the
* query planner's function to determine which partitions will
match. The
* planner is not too careful about freeing memory, so we'll ensure we
* call the function in this context to avoid any memory leaking in the
* executor's memory context.
*/

This is a sloppy cut-and-paste job, not only because somebody changed
one copy of the word "planner" to "executor" and left the others
untouched, but also because the rationale isn't really correct for the
executor anyway, which has memory contexts all over the place and
frees them all the time. I don't know whether the context is not
needed at all or whether the context is needed but the rationale is
different, but I don't buy that explanation.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#148David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#147)
Re: [HACKERS] Runtime Partition Pruning

On 13 April 2018 at 04:57, Robert Haas <robertmhaas@gmail.com> wrote:

BTW, looking at ExecSetupPartitionPruneState:

/*
* Create a sub memory context which we'll use when making calls to the
* query planner's function to determine which partitions will
match. The
* planner is not too careful about freeing memory, so we'll ensure we
* call the function in this context to avoid any memory leaking in the
* executor's memory context.
*/

This is a sloppy cut-and-paste job, not only because somebody changed
one copy of the word "planner" to "executor" and left the others
untouched, but also because the rationale isn't really correct for the
executor anyway, which has memory contexts all over the place and
frees them all the time. I don't know whether the context is not
needed at all or whether the context is needed but the rationale is
different, but I don't buy that explanation.

The comment is written exactly as intended. Unsure which of the
"planner"s you think should be "executor".

The context is needed. I can easily produce an OOM without it.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#149Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Robert Haas (#147)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/13 1:57, Robert Haas wrote:

It might be possible to do something better in each module by keeping
an array indexed by RTI which have each entry NULL initially then on
first relation_open set the element in the array to that pointer.

I'm not sure that makes a lot of sense in the planner, but in the
executor it might be a good idea. See also
/messages/by-id/CA+TgmoYKToP4-adCFFRNrO21OGuH=phx-fiB1dYoqksNYX6YHQ@mail.gmail.com
for related discussion. I think that a coding pattern where we rely
on relation_open(..., NoLock) is inherently dangerous -- it's too easy
to be wrong about whether the lock is sure to have been taken. It
would be much better to open the relation once and hold onto the
pointer, not just for performance reasons, but for robustness.

About the specific relation_open(.., NoLock) under question, I think there
might be a way to address this by opening the tables with the appropriate
lock mode in partitioned_rels list in ExecLockNonleafAppendTables and
keeping the Relation pointers around until ExecEndNode. Then instead of
ExecSetupPartitionPruneState doing relation_open/close(.., NoLock), it
just reuses the one that's passed by the caller.

Attached a PoC patch. David, thoughts?

Thanks,
Amit

Attachments:

open-partitioned-rels-for-pruning-1.patchtext/plain; charset=UTF-8; name=open-partitioned-rels-for-pruning-1.patchDownload
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 11139f743d..e3b3911945 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1244,7 +1244,8 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
  * PartitionPruneInfo.
  */
 PartitionPruneState *
-ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
+ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo,
+							 Relation *partitioned_rels)
 {
 	PartitionPruningData *prunedata;
 	PartitionPruneState *prunestate;
@@ -1303,10 +1304,11 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		pprune->subpart_map = pinfo->subpart_map;
 
 		/*
-		 * Grab some info from the table's relcache; lock was already obtained
-		 * by ExecLockNonLeafAppendTables.
+		 * ExecLockNonLeafAppendTables already opened the relation for us.
 		 */
-		rel = relation_open(pinfo->reloid, NoLock);
+		Assert(partitioned_rels[i] != NULL);
+		rel = partitioned_rels[i];
+		Assert(RelationGetRelid(rel) == pinfo->reloid);
 
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
@@ -1336,8 +1338,6 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		prunestate->execparams = bms_add_members(prunestate->execparams,
 												 pinfo->execparams);
 
-		relation_close(rel, NoLock);
-
 		i++;
 	}
 
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index b963cae730..58a0961eb1 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -858,22 +858,49 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
 /*
  * ExecLockNonLeafAppendTables
  *
- * Locks, if necessary, the tables indicated by the RT indexes contained in
- * the partitioned_rels list.  These are the non-leaf tables in the partition
- * tree controlled by a given Append or MergeAppend node.
+ * Opens and/or locks, if necessary, the tables indicated by the RT indexes
+ * contained in the partitioned_rels list.  These are the non-leaf tables in
+ * the partition tree controlled by a given Append or MergeAppend node.
  */
 void
-ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
+ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels)
 {
 	PlannedStmt *stmt = estate->es_plannedstmt;
 	ListCell   *lc;
+	int			i;
 
+	/*
+	 * If we're going to be performing pruning, allocate space for Relation
+	 * pointers to be used later when setting up partition pruning state in
+	 * ExecSetupPartitionPruneState.
+	 */
+	if (IsA(planstate, AppendState))
+	{
+		AppendState *appendstate = (AppendState *) planstate;
+		Append *node = (Append *) planstate->plan;
+
+		if (node->part_prune_infos != NIL)
+		{
+			Assert(list_length(node->part_prune_infos) ==
+				   list_length(partitioned_rels));
+			appendstate->partitioned_rels = (Relation *)
+								palloc(sizeof(Relation) *
+									   list_length(node->part_prune_infos));
+			appendstate->num_partitioned_rels =
+									   list_length(node->part_prune_infos);
+		}
+	}
+
+	i = 0;
 	foreach(lc, partitioned_rels)
 	{
 		ListCell   *l;
 		Index		rti = lfirst_int(lc);
 		bool		is_result_rel = false;
 		Oid			relid = getrelid(rti, estate->es_range_table);
+		int			lockmode;
 
 		/* If this is a result relation, already locked in InitPlan */
 		foreach(l, stmt->nonleafResultRelations)
@@ -903,9 +930,37 @@ ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
 			}
 
 			if (rc && RowMarkRequiresRowShareLock(rc->markType))
-				LockRelationOid(relid, RowShareLock);
+				lockmode = RowShareLock;
 			else
-				LockRelationOid(relid, AccessShareLock);
+				lockmode = AccessShareLock;
+			switch (nodeTag(planstate))
+			{
+				/*
+				 * For Append, we may need to store the Relation pointers to
+				 * be used later when setting up partition pruning state.
+				 */
+				case T_AppendState:
+					{
+						AppendState *appendstate = (AppendState *) planstate;
+
+						if (appendstate->partitioned_rels)
+							appendstate->partitioned_rels[i] =
+												heap_open(relid, lockmode);
+						else
+							LockRelationOid(relid, lockmode);
+						i++;
+					}
+
+				/* Just lock here; there is no pruning support. */
+				case T_MergeAppendState:
+					LockRelationOid(relid, lockmode);
+					break;
+
+				default:
+					elog(ERROR, "invalid PlanState node: %d",
+						 nodeTag(planstate));
+					break;
+			}
 		}
 	}
 }
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index d062cfddac..833026c000 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -112,30 +112,33 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	Assert(!(eflags & EXEC_FLAG_MARK));
 
 	/*
-	 * Lock the non-leaf tables in the partition tree controlled by this node.
-	 * It's a no-op for non-partitioned parent tables.
-	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
-
-	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
 
+	/*
+	 * Lock the non-leaf tables in the partition tree controlled by this node.
+	 * It's a no-op for non-partitioned parent tables.
+	 */
+	ExecLockNonLeafAppendTables((PlanState *) appendstate, estate,
+								node->partitioned_rels);
+
 	/* Let choose_next_subplan_* function handle setting the first subplan */
 	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
 	/* If run-time partition pruning is enabled, then set that up now */
-	if (node->part_prune_infos != NIL)
+	if (node->part_prune_infos != NULL)
 	{
 		PartitionPruneState *prunestate;
 
-		ExecAssignExprContext(estate, &appendstate->ps);
-
+		/* ExecLockNonLeafAppendTables must have set this up. */
+		Assert(appendstate->partitioned_rels != NULL);
 		prunestate = ExecSetupPartitionPruneState(&appendstate->ps,
-												  node->part_prune_infos);
+												  node->part_prune_infos,
+											appendstate->partitioned_rels);
+		ExecAssignExprContext(estate, &appendstate->ps);
 
 		/*
 		 * When there are external params matching the partition key we may be
@@ -309,6 +312,7 @@ ExecEndAppend(AppendState *node)
 	PlanState **appendplans;
 	int			nplans;
 	int			i;
+	int			num_partitioned_rels;
 
 	/*
 	 * get information from the node
@@ -317,6 +321,15 @@ ExecEndAppend(AppendState *node)
 	nplans = node->as_nplans;
 
 	/*
+	 * Close partitioned rels that we may have opened for partition
+	 * pruning.
+	 */
+	num_partitioned_rels = node->num_partitioned_rels;
+	Assert(node->partitioned_rels != NULL || num_partitioned_rels == 0);
+	for (i = 0; i < num_partitioned_rels; i++)
+		heap_close(node->partitioned_rels[i], NoLock);
+
+	/*
 	 * shut down each of the subscans
 	 */
 	for (i = 0; i < nplans; i++)
diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..6f28002ce2 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -76,7 +76,8 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 * Lock the non-leaf tables in the partition tree controlled by this node.
 	 * It's a no-op for non-partitioned parent tables.
 	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
+	ExecLockNonLeafAppendTables((PlanState *) mergestate, estate,
+								node->partitioned_rels);
 
 	/*
 	 * Set up empty vector of subplan states
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 0c36c8be30..134eb9290f 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -118,6 +118,7 @@ typedef struct PartitionTupleRouting
  * bypass certain subnodes when we have proofs that indicate that no tuple
  * matching the 'pruning_steps' will be found within.
  *
+ * parent						RelationData pointer of the parent
  * subnode_map					An array containing the subnode index which
  *								matches this partition index, or -1 if the
  *								subnode has been pruned already.
@@ -137,6 +138,7 @@ typedef struct PartitionTupleRouting
  */
 typedef struct PartitionPruningData
 {
+	Relation	parent;
 	int		   *subnode_map;
 	int		   *subpart_map;
 	Bitmapset  *present_parts;
@@ -205,7 +207,8 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
 						PartitionTupleRouting *proute);
 extern PartitionPruneState *ExecSetupPartitionPruneState(PlanState *planstate,
-						  List *partitionpruneinfo);
+						  List *partitionpruneinfo,
+						  Relation *partitioned_rels);
 extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
 extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
 								int nsubnodes);
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 45a077a949..6300397099 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -526,7 +526,9 @@ extern void UnregisterExprContextCallback(ExprContext *econtext,
 							  ExprContextCallbackFunction function,
 							  Datum arg);
 
-extern void ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate);
+extern void ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels);
 
 extern Datum GetAttributeByName(HeapTupleHeader tuple, const char *attname,
 				   bool *isNull);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9fe0b79095..b6188f8458 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1089,6 +1089,8 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Relation   *partitioned_rels;
+	int			num_partitioned_rels;	/* number of entries in above array */
 	struct PartitionPruneState *as_prune_state;
 	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
#150Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#149)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/13 14:38, Amit Langote wrote:

About the specific relation_open(.., NoLock) under question, I think there
might be a way to address this by opening the tables with the appropriate
lock mode in partitioned_rels list in ExecLockNonleafAppendTables

That may have sounded a bit confusing:

I meant to say: "by opening the tables in partitioned_rels list with the
appropriate lock mode in ExecLockNonleafAppendTables"

Attached a PoC patch.

There were a couple of unnecessary hunks, which removed in the attached.

Thanks,
Amit

Attachments:

open-partitioned-rels-for-pruning-2.patchtext/plain; charset=UTF-8; name=open-partitioned-rels-for-pruning-2.patchDownload
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 11139f743d..e3b3911945 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1244,7 +1244,8 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
  * PartitionPruneInfo.
  */
 PartitionPruneState *
-ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
+ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo,
+							 Relation *partitioned_rels)
 {
 	PartitionPruningData *prunedata;
 	PartitionPruneState *prunestate;
@@ -1303,10 +1304,11 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		pprune->subpart_map = pinfo->subpart_map;
 
 		/*
-		 * Grab some info from the table's relcache; lock was already obtained
-		 * by ExecLockNonLeafAppendTables.
+		 * ExecLockNonLeafAppendTables already opened the relation for us.
 		 */
-		rel = relation_open(pinfo->reloid, NoLock);
+		Assert(partitioned_rels[i] != NULL);
+		rel = partitioned_rels[i];
+		Assert(RelationGetRelid(rel) == pinfo->reloid);
 
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
@@ -1336,8 +1338,6 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		prunestate->execparams = bms_add_members(prunestate->execparams,
 												 pinfo->execparams);
 
-		relation_close(rel, NoLock);
-
 		i++;
 	}
 
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index b963cae730..58a0961eb1 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -858,22 +858,49 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
 /*
  * ExecLockNonLeafAppendTables
  *
- * Locks, if necessary, the tables indicated by the RT indexes contained in
- * the partitioned_rels list.  These are the non-leaf tables in the partition
- * tree controlled by a given Append or MergeAppend node.
+ * Opens and/or locks, if necessary, the tables indicated by the RT indexes
+ * contained in the partitioned_rels list.  These are the non-leaf tables in
+ * the partition tree controlled by a given Append or MergeAppend node.
  */
 void
-ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
+ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels)
 {
 	PlannedStmt *stmt = estate->es_plannedstmt;
 	ListCell   *lc;
+	int			i;
 
+	/*
+	 * If we're going to be performing pruning, allocate space for Relation
+	 * pointers to be used later when setting up partition pruning state in
+	 * ExecSetupPartitionPruneState.
+	 */
+	if (IsA(planstate, AppendState))
+	{
+		AppendState *appendstate = (AppendState *) planstate;
+		Append *node = (Append *) planstate->plan;
+
+		if (node->part_prune_infos != NIL)
+		{
+			Assert(list_length(node->part_prune_infos) ==
+				   list_length(partitioned_rels));
+			appendstate->partitioned_rels = (Relation *)
+								palloc(sizeof(Relation) *
+									   list_length(node->part_prune_infos));
+			appendstate->num_partitioned_rels =
+									   list_length(node->part_prune_infos);
+		}
+	}
+
+	i = 0;
 	foreach(lc, partitioned_rels)
 	{
 		ListCell   *l;
 		Index		rti = lfirst_int(lc);
 		bool		is_result_rel = false;
 		Oid			relid = getrelid(rti, estate->es_range_table);
+		int			lockmode;
 
 		/* If this is a result relation, already locked in InitPlan */
 		foreach(l, stmt->nonleafResultRelations)
@@ -903,9 +930,37 @@ ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
 			}
 
 			if (rc && RowMarkRequiresRowShareLock(rc->markType))
-				LockRelationOid(relid, RowShareLock);
+				lockmode = RowShareLock;
 			else
-				LockRelationOid(relid, AccessShareLock);
+				lockmode = AccessShareLock;
+			switch (nodeTag(planstate))
+			{
+				/*
+				 * For Append, we may need to store the Relation pointers to
+				 * be used later when setting up partition pruning state.
+				 */
+				case T_AppendState:
+					{
+						AppendState *appendstate = (AppendState *) planstate;
+
+						if (appendstate->partitioned_rels)
+							appendstate->partitioned_rels[i] =
+												heap_open(relid, lockmode);
+						else
+							LockRelationOid(relid, lockmode);
+						i++;
+					}
+
+				/* Just lock here; there is no pruning support. */
+				case T_MergeAppendState:
+					LockRelationOid(relid, lockmode);
+					break;
+
+				default:
+					elog(ERROR, "invalid PlanState node: %d",
+						 nodeTag(planstate));
+					break;
+			}
 		}
 	}
 }
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index d062cfddac..0799bbc919 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -112,18 +112,19 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	Assert(!(eflags & EXEC_FLAG_MARK));
 
 	/*
-	 * Lock the non-leaf tables in the partition tree controlled by this node.
-	 * It's a no-op for non-partitioned parent tables.
-	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
-
-	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
 
+	/*
+	 * Lock the non-leaf tables in the partition tree controlled by this node.
+	 * It's a no-op for non-partitioned parent tables.
+	 */
+	ExecLockNonLeafAppendTables((PlanState *) appendstate, estate,
+								node->partitioned_rels);
+
 	/* Let choose_next_subplan_* function handle setting the first subplan */
 	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
@@ -134,8 +135,11 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 
 		ExecAssignExprContext(estate, &appendstate->ps);
 
+		/* ExecLockNonLeafAppendTables must have set this up. */
+		Assert(appendstate->partitioned_rels != NULL);
 		prunestate = ExecSetupPartitionPruneState(&appendstate->ps,
-												  node->part_prune_infos);
+												  node->part_prune_infos,
+											appendstate->partitioned_rels);
 
 		/*
 		 * When there are external params matching the partition key we may be
@@ -309,6 +313,7 @@ ExecEndAppend(AppendState *node)
 	PlanState **appendplans;
 	int			nplans;
 	int			i;
+	int			num_partitioned_rels;
 
 	/*
 	 * get information from the node
@@ -317,6 +322,15 @@ ExecEndAppend(AppendState *node)
 	nplans = node->as_nplans;
 
 	/*
+	 * Close partitioned rels that we may have opened for partition
+	 * pruning.
+	 */
+	num_partitioned_rels = node->num_partitioned_rels;
+	Assert(node->partitioned_rels != NULL || num_partitioned_rels == 0);
+	for (i = 0; i < num_partitioned_rels; i++)
+		heap_close(node->partitioned_rels[i], NoLock);
+
+	/*
 	 * shut down each of the subscans
 	 */
 	for (i = 0; i < nplans; i++)
diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..6f28002ce2 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -76,7 +76,8 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 * Lock the non-leaf tables in the partition tree controlled by this node.
 	 * It's a no-op for non-partitioned parent tables.
 	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
+	ExecLockNonLeafAppendTables((PlanState *) mergestate, estate,
+								node->partitioned_rels);
 
 	/*
 	 * Set up empty vector of subplan states
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 0c36c8be30..134eb9290f 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -118,6 +118,7 @@ typedef struct PartitionTupleRouting
  * bypass certain subnodes when we have proofs that indicate that no tuple
  * matching the 'pruning_steps' will be found within.
  *
+ * parent						RelationData pointer of the parent
  * subnode_map					An array containing the subnode index which
  *								matches this partition index, or -1 if the
  *								subnode has been pruned already.
@@ -137,6 +138,7 @@ typedef struct PartitionTupleRouting
  */
 typedef struct PartitionPruningData
 {
+	Relation	parent;
 	int		   *subnode_map;
 	int		   *subpart_map;
 	Bitmapset  *present_parts;
@@ -205,7 +207,8 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
 						PartitionTupleRouting *proute);
 extern PartitionPruneState *ExecSetupPartitionPruneState(PlanState *planstate,
-						  List *partitionpruneinfo);
+						  List *partitionpruneinfo,
+						  Relation *partitioned_rels);
 extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
 extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
 								int nsubnodes);
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 45a077a949..6300397099 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -526,7 +526,9 @@ extern void UnregisterExprContextCallback(ExprContext *econtext,
 							  ExprContextCallbackFunction function,
 							  Datum arg);
 
-extern void ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate);
+extern void ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels);
 
 extern Datum GetAttributeByName(HeapTupleHeader tuple, const char *attname,
 				   bool *isNull);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9fe0b79095..b6188f8458 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1089,6 +1089,8 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Relation   *partitioned_rels;
+	int			num_partitioned_rels;	/* number of entries in above array */
 	struct PartitionPruneState *as_prune_state;
 	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
#151Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: Amit Langote (#150)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 2018/04/13 14:48, Amit Langote wrote:

On 2018/04/13 14:38, Amit Langote wrote:

About the specific relation_open(.., NoLock) under question, I think there
might be a way to address this by opening the tables with the appropriate
lock mode in partitioned_rels list in ExecLockNonleafAppendTables

That may have sounded a bit confusing:

I meant to say: "by opening the tables in partitioned_rels list with the
appropriate lock mode in ExecLockNonleafAppendTables"

Attached a PoC patch.

There were a couple of unnecessary hunks, which removed in the attached.

Sorry, still a couple more were unnecessary.

Thanks,
Amit

Attachments:

open-partitioned-rels-for-pruning-3.patchtext/plain; charset=UTF-8; name=open-partitioned-rels-for-pruning-3.patchDownload
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 11139f743d..e3b3911945 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1244,7 +1244,8 @@ adjust_partition_tlist(List *tlist, TupleConversionMap *map)
  * PartitionPruneInfo.
  */
 PartitionPruneState *
-ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
+ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo,
+							 Relation *partitioned_rels)
 {
 	PartitionPruningData *prunedata;
 	PartitionPruneState *prunestate;
@@ -1303,10 +1304,11 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		pprune->subpart_map = pinfo->subpart_map;
 
 		/*
-		 * Grab some info from the table's relcache; lock was already obtained
-		 * by ExecLockNonLeafAppendTables.
+		 * ExecLockNonLeafAppendTables already opened the relation for us.
 		 */
-		rel = relation_open(pinfo->reloid, NoLock);
+		Assert(partitioned_rels[i] != NULL);
+		rel = partitioned_rels[i];
+		Assert(RelationGetRelid(rel) == pinfo->reloid);
 
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
@@ -1336,8 +1338,6 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		prunestate->execparams = bms_add_members(prunestate->execparams,
 												 pinfo->execparams);
 
-		relation_close(rel, NoLock);
-
 		i++;
 	}
 
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index b963cae730..58a0961eb1 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -858,22 +858,49 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
 /*
  * ExecLockNonLeafAppendTables
  *
- * Locks, if necessary, the tables indicated by the RT indexes contained in
- * the partitioned_rels list.  These are the non-leaf tables in the partition
- * tree controlled by a given Append or MergeAppend node.
+ * Opens and/or locks, if necessary, the tables indicated by the RT indexes
+ * contained in the partitioned_rels list.  These are the non-leaf tables in
+ * the partition tree controlled by a given Append or MergeAppend node.
  */
 void
-ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
+ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels)
 {
 	PlannedStmt *stmt = estate->es_plannedstmt;
 	ListCell   *lc;
+	int			i;
 
+	/*
+	 * If we're going to be performing pruning, allocate space for Relation
+	 * pointers to be used later when setting up partition pruning state in
+	 * ExecSetupPartitionPruneState.
+	 */
+	if (IsA(planstate, AppendState))
+	{
+		AppendState *appendstate = (AppendState *) planstate;
+		Append *node = (Append *) planstate->plan;
+
+		if (node->part_prune_infos != NIL)
+		{
+			Assert(list_length(node->part_prune_infos) ==
+				   list_length(partitioned_rels));
+			appendstate->partitioned_rels = (Relation *)
+								palloc(sizeof(Relation) *
+									   list_length(node->part_prune_infos));
+			appendstate->num_partitioned_rels =
+									   list_length(node->part_prune_infos);
+		}
+	}
+
+	i = 0;
 	foreach(lc, partitioned_rels)
 	{
 		ListCell   *l;
 		Index		rti = lfirst_int(lc);
 		bool		is_result_rel = false;
 		Oid			relid = getrelid(rti, estate->es_range_table);
+		int			lockmode;
 
 		/* If this is a result relation, already locked in InitPlan */
 		foreach(l, stmt->nonleafResultRelations)
@@ -903,9 +930,37 @@ ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate)
 			}
 
 			if (rc && RowMarkRequiresRowShareLock(rc->markType))
-				LockRelationOid(relid, RowShareLock);
+				lockmode = RowShareLock;
 			else
-				LockRelationOid(relid, AccessShareLock);
+				lockmode = AccessShareLock;
+			switch (nodeTag(planstate))
+			{
+				/*
+				 * For Append, we may need to store the Relation pointers to
+				 * be used later when setting up partition pruning state.
+				 */
+				case T_AppendState:
+					{
+						AppendState *appendstate = (AppendState *) planstate;
+
+						if (appendstate->partitioned_rels)
+							appendstate->partitioned_rels[i] =
+												heap_open(relid, lockmode);
+						else
+							LockRelationOid(relid, lockmode);
+						i++;
+					}
+
+				/* Just lock here; there is no pruning support. */
+				case T_MergeAppendState:
+					LockRelationOid(relid, lockmode);
+					break;
+
+				default:
+					elog(ERROR, "invalid PlanState node: %d",
+						 nodeTag(planstate));
+					break;
+			}
 		}
 	}
 }
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index d062cfddac..0799bbc919 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -112,18 +112,19 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 	Assert(!(eflags & EXEC_FLAG_MARK));
 
 	/*
-	 * Lock the non-leaf tables in the partition tree controlled by this node.
-	 * It's a no-op for non-partitioned parent tables.
-	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
-
-	/*
 	 * create new AppendState for our append node
 	 */
 	appendstate->ps.plan = (Plan *) node;
 	appendstate->ps.state = estate;
 	appendstate->ps.ExecProcNode = ExecAppend;
 
+	/*
+	 * Lock the non-leaf tables in the partition tree controlled by this node.
+	 * It's a no-op for non-partitioned parent tables.
+	 */
+	ExecLockNonLeafAppendTables((PlanState *) appendstate, estate,
+								node->partitioned_rels);
+
 	/* Let choose_next_subplan_* function handle setting the first subplan */
 	appendstate->as_whichplan = INVALID_SUBPLAN_INDEX;
 
@@ -134,8 +135,11 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
 
 		ExecAssignExprContext(estate, &appendstate->ps);
 
+		/* ExecLockNonLeafAppendTables must have set this up. */
+		Assert(appendstate->partitioned_rels != NULL);
 		prunestate = ExecSetupPartitionPruneState(&appendstate->ps,
-												  node->part_prune_infos);
+												  node->part_prune_infos,
+											appendstate->partitioned_rels);
 
 		/*
 		 * When there are external params matching the partition key we may be
@@ -309,6 +313,7 @@ ExecEndAppend(AppendState *node)
 	PlanState **appendplans;
 	int			nplans;
 	int			i;
+	int			num_partitioned_rels;
 
 	/*
 	 * get information from the node
@@ -317,6 +322,15 @@ ExecEndAppend(AppendState *node)
 	nplans = node->as_nplans;
 
 	/*
+	 * Close partitioned rels that we may have opened for partition
+	 * pruning.
+	 */
+	num_partitioned_rels = node->num_partitioned_rels;
+	Assert(node->partitioned_rels != NULL || num_partitioned_rels == 0);
+	for (i = 0; i < num_partitioned_rels; i++)
+		heap_close(node->partitioned_rels[i], NoLock);
+
+	/*
 	 * shut down each of the subscans
 	 */
 	for (i = 0; i < nplans; i++)
diff --git a/src/backend/executor/nodeMergeAppend.c b/src/backend/executor/nodeMergeAppend.c
index 118f4ef07d..6f28002ce2 100644
--- a/src/backend/executor/nodeMergeAppend.c
+++ b/src/backend/executor/nodeMergeAppend.c
@@ -76,7 +76,8 @@ ExecInitMergeAppend(MergeAppend *node, EState *estate, int eflags)
 	 * Lock the non-leaf tables in the partition tree controlled by this node.
 	 * It's a no-op for non-partitioned parent tables.
 	 */
-	ExecLockNonLeafAppendTables(node->partitioned_rels, estate);
+	ExecLockNonLeafAppendTables((PlanState *) mergestate, estate,
+								node->partitioned_rels);
 
 	/*
 	 * Set up empty vector of subplan states
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index 0c36c8be30..9a78f59347 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -205,7 +205,8 @@ extern HeapTuple ConvertPartitionTupleSlot(TupleConversionMap *map,
 extern void ExecCleanupTupleRouting(ModifyTableState *mtstate,
 						PartitionTupleRouting *proute);
 extern PartitionPruneState *ExecSetupPartitionPruneState(PlanState *planstate,
-						  List *partitionpruneinfo);
+						  List *partitionpruneinfo,
+						  Relation *partitioned_rels);
 extern Bitmapset *ExecFindMatchingSubPlans(PartitionPruneState *prunestate);
 extern Bitmapset *ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate,
 								int nsubnodes);
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 45a077a949..6300397099 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -526,7 +526,9 @@ extern void UnregisterExprContextCallback(ExprContext *econtext,
 							  ExprContextCallbackFunction function,
 							  Datum arg);
 
-extern void ExecLockNonLeafAppendTables(List *partitioned_rels, EState *estate);
+extern void ExecLockNonLeafAppendTables(PlanState *planstate,
+							EState *estate,
+							List *partitioned_rels);
 
 extern Datum GetAttributeByName(HeapTupleHeader tuple, const char *attname,
 				   bool *isNull);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9fe0b79095..b6188f8458 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1089,6 +1089,8 @@ struct AppendState
 	int			as_whichplan;
 	ParallelAppendState *as_pstate; /* parallel coordination info */
 	Size		pstate_len;		/* size of parallel coordination info */
+	Relation   *partitioned_rels;
+	int			num_partitioned_rels;	/* number of entries in above array */
 	struct PartitionPruneState *as_prune_state;
 	Bitmapset  *as_valid_subplans;
 	bool		(*choose_next_subplan) (AppendState *);
#152Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#148)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Apr 12, 2018 at 6:01 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

On 13 April 2018 at 04:57, Robert Haas <robertmhaas@gmail.com> wrote:

BTW, looking at ExecSetupPartitionPruneState:

/*
* Create a sub memory context which we'll use when making calls to the
* query planner's function to determine which partitions will
match. The
* planner is not too careful about freeing memory, so we'll ensure we
* call the function in this context to avoid any memory leaking in the
* executor's memory context.
*/

This is a sloppy cut-and-paste job, not only because somebody changed
one copy of the word "planner" to "executor" and left the others
untouched, but also because the rationale isn't really correct for the
executor anyway, which has memory contexts all over the place and
frees them all the time. I don't know whether the context is not
needed at all or whether the context is needed but the rationale is
different, but I don't buy that explanation.

The comment is written exactly as intended. Unsure which of the
"planner"s you think should be "executor".

The context is needed. I can easily produce an OOM without it.

Oh, crap. You know, I totally misread what that comment was trying to
say. Sorry.

But I wonder why it's the executor's job to clean up after the
planner, instead of adjusting the relevant planner functions to avoid
leaking memory?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#153David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#152)
Re: [HACKERS] Runtime Partition Pruning

On 14 April 2018 at 05:04, Robert Haas <robertmhaas@gmail.com> wrote:

But I wonder why it's the executor's job to clean up after the
planner, instead of adjusting the relevant planner functions to avoid
leaking memory?

It might be possible, but it might also be risky and difficult.

For a while, during my review of the faster partition pruning patch I
was asking Amit to add pfree() calls in various places for this exact
reason, but in the end, I gave up and decided it was easier to just
create a new memory context to call the planner function from. I've
now forgotten the exact reason why I finally decided it was too much
trouble. The pruning code now works using your step logic so perhaps
that reason no longer applies, although, on a quick scan of the
pruning code now, it seems to require that get_matching_partitions
performs a deep pfree of each PruneStepResult. However, there is still
partkey_datum_from_expr which performs ExecInitExpr, although perhaps
that can just be done once, and the result stashed in the
PartitionPruneContext.

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#154Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#153)
Re: [HACKERS] Runtime Partition Pruning

David Rowley wrote:

For a while, during my review of the faster partition pruning patch I
was asking Amit to add pfree() calls in various places for this exact
reason, but in the end, I gave up and decided it was easier to just
create a new memory context to call the planner function from. I've
now forgotten the exact reason why I finally decided it was too much
trouble. The pruning code now works using your step logic so perhaps
that reason no longer applies, although, on a quick scan of the
pruning code now, it seems to require that get_matching_partitions
performs a deep pfree of each PruneStepResult. However, there is still
partkey_datum_from_expr which performs ExecInitExpr, although perhaps
that can just be done once, and the result stashed in the
PartitionPruneContext.

I think trying to replace a well-placed MemoryContextReset (or Delete)
with a bunch of individual pfrees is pointless.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#155David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#154)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 17 April 2018 at 14:33, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

David Rowley wrote:

For a while, during my review of the faster partition pruning patch I
was asking Amit to add pfree() calls in various places for this exact
reason, but in the end, I gave up and decided it was easier to just
create a new memory context to call the planner function from. I've
now forgotten the exact reason why I finally decided it was too much
trouble. The pruning code now works using your step logic so perhaps
that reason no longer applies, although, on a quick scan of the
pruning code now, it seems to require that get_matching_partitions
performs a deep pfree of each PruneStepResult. However, there is still
partkey_datum_from_expr which performs ExecInitExpr, although perhaps
that can just be done once, and the result stashed in the
PartitionPruneContext.

I think trying to replace a well-placed MemoryContextReset (or Delete)
with a bunch of individual pfrees is pointless.

I agree. I think I'd sleep better at night with the context reset in
there rather than hoping we've managed to pfree everything.

I did go and start working on a patch to test how possible this would
be and came up with the attached. I've left a stray
MemoryContextStatsDetail call in there which does indicate that
something is not being freed. I'm just not sure what it is yet.

The patch does happen to improve performance slightly, but that is
most likely due to the caching of the ExprStates rather than the
change of memory management. It's not really possible to do that with
the reset unless we stored the executor's memory context in
PartitionPruneContext and did a context switch back inside
partkey_datum_from_expr before calling ExecInitExpr.

My test case was as follows:

create table p (a int, value int) partition by hash (a);
select 'create table p'||x|| ' partition of p for values with (modulus
10, remainder '||x||');' from generate_series(0,9) x;
\gexec
create table t1 (a int);

insert into p select x,x from generate_Series(1,10000000) x;
insert into t1 select x from generate_series(1,10000000) x;

create index on p(a);

set enable_hashjoin = 0;
set enable_mergejoin = 0;
explain analyze select count(*) from t1 inner join p on t1.a=p.a;

-- Unpatched
Execution Time: 19725.981 ms
Execution Time: 19533.655 ms
Execution Time: 19542.854 ms

-- Patched
Execution Time: 17389.537 ms
Execution Time: 17603.802 ms
Execution Time: 17618.670 ms

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

recycle_mem_part_prune.patchapplication/octet-stream; name=recycle_mem_part_prune.patchDownload
diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 23a74bc3d9..d06b718c64 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1438,18 +1438,6 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 	prunestate->extparams = NULL;
 	prunestate->execparams = NULL;
 
-	/*
-	 * Create a sub memory context which we'll use when making calls to the
-	 * query planner's function to determine which partitions will match.  The
-	 * planner is not too careful about freeing memory, so we'll ensure we
-	 * call the function in this context to avoid any memory leaking in the
-	 * executor's memory context.
-	 */
-	prunestate->prune_context =
-		AllocSetContextCreate(CurrentMemoryContext,
-							  "Partition Prune",
-							  ALLOCSET_DEFAULT_SIZES);
-
 	i = 0;
 	foreach(lc, partitionpruneinfo)
 	{
@@ -1493,6 +1481,8 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
 		context->planstate = planstate;
 		context->safeparams = NULL; /* empty for now */
+		context->exprstates = palloc0(sizeof(ExprState *) *
+									  list_length(pinfo->pruning_steps));
 
 		pprune->pruning_steps = pinfo->pruning_steps;
 		pprune->extparams = bms_copy(pinfo->extparams);
@@ -1546,7 +1536,6 @@ Bitmapset *
 ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate, int nsubnodes)
 {
 	PartitionPruningData *pprune;
-	MemoryContext oldcontext;
 	Bitmapset  *result = NULL;
 
 	/*
@@ -1557,22 +1546,9 @@ ExecFindInitialMatchingSubPlans(PartitionPruneState *prunestate, int nsubnodes)
 
 	pprune = prunestate->partprunedata;
 
-	/*
-	 * Switch to a temp context to avoid leaking memory in the executor's
-	 * memory context.
-	 */
-	oldcontext = MemoryContextSwitchTo(prunestate->prune_context);
-
 	/* Determine which subnodes match the external params */
 	find_subplans_for_params_recurse(prunestate, pprune, false, &result);
 
-	MemoryContextSwitchTo(oldcontext);
-
-	/* Move to the correct memory context */
-	result = bms_copy(result);
-
-	MemoryContextReset(prunestate->prune_context);
-
 	/*
 	 * Record that partition pruning has been performed for external params.
 	 * These are not required again afterwards, and nullifying them helps
@@ -1670,26 +1646,12 @@ Bitmapset *
 ExecFindMatchingSubPlans(PartitionPruneState *prunestate)
 {
 	PartitionPruningData *pprune;
-	MemoryContext oldcontext;
 	Bitmapset  *result = NULL;
 
 	pprune = prunestate->partprunedata;
 
-	/*
-	 * Switch to a temp context to avoid leaking memory in the executor's
-	 * memory context.
-	 */
-	oldcontext = MemoryContextSwitchTo(prunestate->prune_context);
-
 	find_subplans_for_params_recurse(prunestate, pprune, true, &result);
 
-	MemoryContextSwitchTo(oldcontext);
-
-	/* Move to the correct memory context */
-	result = bms_copy(result);
-
-	MemoryContextReset(prunestate->prune_context);
-
 	return result;
 }
 
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index d062cfddac..144b8651a3 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -321,6 +321,8 @@ ExecEndAppend(AppendState *node)
 	 */
 	for (i = 0; i < nplans; i++)
 		ExecEndNode(appendplans[i]);
+
+	MemoryContextStatsDetail(TopMemoryContext, INT_MAX);
 }
 
 void
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 7666c6c412..d1a90b05fb 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -169,7 +169,7 @@ static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *cont
 static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 							   Expr *partkey, Expr **outconst);
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
-						Expr *expr, Datum *value);
+						int step_id, Expr *expr, Datum *value);
 
 /*
  * make_partition_pruneinfo
@@ -444,6 +444,7 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	/* Not valid when being called from the planner */
 	context.planstate = NULL;
 	context.safeparams = NULL;
+	context.exprstates = NULL;
 
 	/* Actual pruning happens here. */
 	partindexes = get_matching_partitions(&context, pruning_steps);
@@ -463,6 +464,10 @@ prune_append_rel_partitions(RelOptInfo *rel)
  *
  * Returns a Bitmapset of the RelOptInfo->part_rels indexes of the surviving
  * partitions.
+ *
+ * Caution: This is used inside the executor so we must be careful here to
+ * ensure we free all memory allocated within this function and any functions
+ * called from here.
  */
 Bitmapset *
 get_matching_partitions(PartitionPruneContext *context, List *pruning_steps)
@@ -552,6 +557,14 @@ get_matching_partitions(PartitionPruneContext *context, List *pruning_steps)
 		result = bms_add_member(result, context->boundinfo->default_index);
 	}
 
+	/* Ensure we free all allocations */
+	for (i = 0; i < num_steps; i++)
+	{
+		bms_free(results[i]->bound_offsets);
+		pfree(results[i]);
+	}
+	pfree(results);
+
 	return result;
 }
 
@@ -2785,7 +2798,8 @@ perform_pruning_base_step(PartitionPruneContext *context,
 			Datum		datum;
 
 			expr = lfirst(lc1);
-			if (partkey_datum_from_expr(context, expr, &datum))
+			if (partkey_datum_from_expr(context, opstep->step.step_id, expr,
+										&datum))
 			{
 				Oid			cmpfn;
 
@@ -3023,7 +3037,7 @@ match_boolean_partition_clause(Oid partopfamily, Expr *clause, Expr *partkey,
  *		evaluation was possible, otherwise false.
  */
 static bool
-partkey_datum_from_expr(PartitionPruneContext *context,
+partkey_datum_from_expr(PartitionPruneContext *context, int step_id,
 						Expr *expr, Datum *value)
 {
 	switch (nodeTag(expr))
@@ -3041,12 +3055,15 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 			if (context->planstate &&
 				bms_is_member(((Param *) expr)->paramid, context->safeparams))
 			{
-				ExprState  *exprstate;
 				bool		isNull;
 
-				exprstate = ExecInitExpr(expr, context->planstate);
+				Assert(context->exprstates);
+
+				if (!context->exprstates[step_id])
+					context->exprstates[step_id] =
+									ExecInitExpr(expr, context->planstate);
 
-				*value = ExecEvalExprSwitchContext(exprstate,
+				*value = ExecEvalExprSwitchContext(context->exprstates[step_id],
 												   context->planstate->ps_ExprContext,
 												   &isNull);
 				if (isNull)
diff --git a/src/include/executor/execPartition.h b/src/include/executor/execPartition.h
index e81bdc4a0a..4c7653e66f 100644
--- a/src/include/executor/execPartition.h
+++ b/src/include/executor/execPartition.h
@@ -159,8 +159,6 @@ typedef struct PartitionPruningData
  *						partitioned relation. First element contains the
  *						details for the target partitioned table.
  * num_partprunedata	Number of items in 'partprunedata' array.
- * 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
  *						partition key in each of the contained
  *						PartitionPruningData structs.
@@ -173,7 +171,6 @@ typedef struct PartitionPruneState
 {
 	PartitionPruningData *partprunedata;
 	int			num_partprunedata;
-	MemoryContext prune_context;
 	Bitmapset  *extparams;
 	Bitmapset  *execparams;
 	Bitmapset  *allparams;
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index a5568abce6..9f5fd198e7 100644
--- a/src/include/partitioning/partprune.h
+++ b/src/include/partitioning/partprune.h
@@ -50,6 +50,12 @@ typedef struct PartitionPruneContext
 	 * are not safe to use until the executor is running.
 	 */
 	Bitmapset  *safeparams;
+
+	/*
+	 * Array of ExprStates, one per pruning step.  Must be allocated if
+	 * planstate is non-NULL, otherwise can be NULL
+	 */
+	ExprState **exprstates;
 } PartitionPruneContext;
 
 
#156Robert Haas
robertmhaas@gmail.com
In reply to: David Rowley (#155)
Re: [HACKERS] Runtime Partition Pruning

On Mon, Apr 16, 2018 at 10:46 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

I did go and start working on a patch to test how possible this would
be and came up with the attached. I've left a stray
MemoryContextStatsDetail call in there which does indicate that
something is not being freed. I'm just not sure what it is yet.

The patch does happen to improve performance slightly, but that is
most likely due to the caching of the ExprStates rather than the
change of memory management. It's not really possible to do that with
the reset unless we stored the executor's memory context in
PartitionPruneContext and did a context switch back inside
partkey_datum_from_expr before calling ExecInitExpr.

10% is more than a "slight" improvement, I'd say! It's certainly got
to be worth avoiding the repeated calls to ExecInitExpr, whatever we
do about the memory contexts.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#157David Rowley
david.rowley@2ndquadrant.com
In reply to: Robert Haas (#156)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

On 19 April 2018 at 03:13, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Apr 16, 2018 at 10:46 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

I did go and start working on a patch to test how possible this would
be and came up with the attached. I've left a stray
MemoryContextStatsDetail call in there which does indicate that
something is not being freed. I'm just not sure what it is yet.

The patch does happen to improve performance slightly, but that is
most likely due to the caching of the ExprStates rather than the
change of memory management. It's not really possible to do that with
the reset unless we stored the executor's memory context in
PartitionPruneContext and did a context switch back inside
partkey_datum_from_expr before calling ExecInitExpr.

10% is more than a "slight" improvement, I'd say! It's certainly got
to be worth avoiding the repeated calls to ExecInitExpr, whatever we
do about the memory contexts.

I've attached a patch which does just this. On benchmarking again with
this single change performance has improved 15% over master.

Also, out of curiosity, I also checked what this performed like before
the run-time pruning patch was committed (5c0675215). Taking the
average of the times below, it seems without this patch the
performance of this case has improved about 356% and about 410% with
this patch. So, I agree, it might be worth considering.

create table p (a int, value int) partition by hash (a);
select 'create table p'||x|| ' partition of p for values with (modulus
10, remainder '||x||');' from generate_series(0,9) x;
\gexec
create table t1 (a int);

insert into p select x,x from generate_Series(1,1000) x;
insert into t1 select x from generate_series(1,1000) x;

create index on p(a);

set enable_hashjoin = 0;
set enable_mergejoin = 0;
explain analyze select count(*) from t1 inner join p on t1.a=p.a;

-- Unpatched
Execution Time: 20413.975 ms
Execution Time: 20232.050 ms
Execution Time: 20229.116 ms

-- Patched
Execution Time: 17758.111 ms
Execution Time: 17645.151 ms
Execution Time: 17492.260 ms

-- 5c0675215e153ba1297fd494b34af2fdebd645d1
Execution Time: 72875.161 ms
Execution Time: 71817.757 ms
Execution Time: 72411.730 ms

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

Attachments:

0001-Initialize-expr-states-once-in-run-time-partition-pr.patchapplication/octet-stream; name=0001-Initialize-expr-states-once-in-run-time-partition-pr.patchDownload
From 5a989e94e8d7e50a331740f7e7d2ccffed4e7391 Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 19 Apr 2018 11:51:16 +1200
Subject: [PATCH] Initialize expr states once in run-time partition pruning

Previously ExecInitExpr was called every time a Param needed to be evaulated
during run-time partition pruning.  This meant additional memory allocations
were made.  Here we change things so we now call this function just once
during the setup of run-time pruning.  We cache the result in
PartitionPruneContext so that it can be re-used each time pruning is invoked.
---
 src/backend/executor/execPartition.c | 31 +++++++++++++++++++++++++++++++
 src/backend/partitioning/partprune.c | 17 ++++++++++++-----
 src/include/partitioning/partprune.h |  6 ++++++
 3 files changed, 49 insertions(+), 5 deletions(-)

diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index 23a74bc3d9..5e4e24683c 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1459,7 +1459,9 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		PartitionDesc partdesc;
 		Relation	rel;
 		PartitionKey partkey;
+		ListCell   *lc2;
 		int			partnatts;
+		int			n_steps;
 
 		pprune->present_parts = bms_copy(pinfo->present_parts);
 		pprune->subnode_map = palloc(sizeof(int) * pinfo->nparts);
@@ -1482,6 +1484,7 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
+		n_steps = list_length(pinfo->pruning_steps);
 
 		context->strategy = partkey->strategy;
 		context->partnatts = partnatts = partkey->partnatts;
@@ -1493,6 +1496,34 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
 		context->planstate = planstate;
 		context->safeparams = NULL; /* empty for now */
+		context->exprstates = palloc0(sizeof(ExprState *) * n_steps * partnatts);
+
+		/* Initialize expression states for each expression */
+		foreach(lc2, pinfo->pruning_steps)
+		{
+			PartitionPruneStepOp *step = (PartitionPruneStepOp *) lfirst(lc2);
+			ListCell   *lc3;
+			int			keyno;
+
+			if (!IsA(step, PartitionPruneStepOp))
+				continue;
+
+			Assert(list_length(step->exprs) <= partnatts);
+
+			keyno = 0;
+			foreach(lc3, step->exprs)
+			{
+				Expr	   *expr = (Expr *) lfirst(lc3);
+
+				/*
+				 * partkey_datum_from_expr does not need an expression state
+				 * to evaluate a Const.
+				 */
+				if (!IsA(expr, Const))
+					context->exprstates[step->step.step_id * partnatts + keyno] = ExecInitExpr(expr, context->planstate);
+				keyno++;
+			}
+		}
 
 		pprune->pruning_steps = pinfo->pruning_steps;
 		pprune->extparams = bms_copy(pinfo->extparams);
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index 7666c6c412..f688357fe4 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -169,7 +169,8 @@ static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *cont
 static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 							   Expr *partkey, Expr **outconst);
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
-						Expr *expr, Datum *value);
+						int step_id, int keyno, int partnatts, Expr *expr,
+						Datum *value);
 
 /*
  * make_partition_pruneinfo
@@ -444,6 +445,7 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	/* Not valid when being called from the planner */
 	context.planstate = NULL;
 	context.safeparams = NULL;
+	context.exprstates = NULL;
 
 	/* Actual pruning happens here. */
 	partindexes = get_matching_partitions(&context, pruning_steps);
@@ -2785,7 +2787,8 @@ perform_pruning_base_step(PartitionPruneContext *context,
 			Datum		datum;
 
 			expr = lfirst(lc1);
-			if (partkey_datum_from_expr(context, expr, &datum))
+			if (partkey_datum_from_expr(context, opstep->step.step_id, keyno,
+										context->partnatts, expr, &datum))
 			{
 				Oid			cmpfn;
 
@@ -3023,8 +3026,8 @@ match_boolean_partition_clause(Oid partopfamily, Expr *clause, Expr *partkey,
  *		evaluation was possible, otherwise false.
  */
 static bool
-partkey_datum_from_expr(PartitionPruneContext *context,
-						Expr *expr, Datum *value)
+partkey_datum_from_expr(PartitionPruneContext * context, int step_id,
+						int keyno, int partnatts, Expr *expr, Datum *value)
 {
 	switch (nodeTag(expr))
 	{
@@ -3044,7 +3047,11 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 				ExprState  *exprstate;
 				bool		isNull;
 
-				exprstate = ExecInitExpr(expr, context->planstate);
+				Assert(context->exprstates);
+
+				exprstate = context->exprstates[step_id * partnatts + keyno];
+
+				Assert(exprstate);
 
 				*value = ExecEvalExprSwitchContext(exprstate,
 												   context->planstate->ps_ExprContext,
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index a5568abce6..31a4592397 100644
--- a/src/include/partitioning/partprune.h
+++ b/src/include/partitioning/partprune.h
@@ -50,6 +50,12 @@ typedef struct PartitionPruneContext
 	 * are not safe to use until the executor is running.
 	 */
 	Bitmapset  *safeparams;
+
+	/*
+	 * Array of ExprStates, one for each partkey in each pruning step.  Must
+	 * be allocated if planstate is non-NULL, otherwise can be NULL.
+	 */
+	ExprState **exprstates;
 } PartitionPruneContext;
 
 
-- 
2.16.2.windows.1

#158David Rowley
david.rowley@2ndquadrant.com
In reply to: David Rowley (#157)
Re: [HACKERS] Runtime Partition Pruning

On 19 April 2018 at 12:04, David Rowley <david.rowley@2ndquadrant.com> wrote:

insert into p select x,x from generate_Series(1,1000) x;
insert into t1 select x from generate_series(1,1000) x;

Correction. These were meant to read:

insert into p select x,x from generate_Series(1,10000000) x;
insert into t1 select x from generate_series(1,10000000) x;

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#159Amit Langote
Langote_Amit_f8@lab.ntt.co.jp
In reply to: David Rowley (#157)
Re: [HACKERS] Runtime Partition Pruning

Hi David.

On 2018/04/19 9:04, David Rowley wrote:

On 19 April 2018 at 03:13, Robert Haas <robertmhaas@gmail.com> wrote:

On Mon, Apr 16, 2018 at 10:46 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:

The patch does happen to improve performance slightly, but that is
most likely due to the caching of the ExprStates rather than the
change of memory management. It's not really possible to do that with
the reset unless we stored the executor's memory context in
PartitionPruneContext and did a context switch back inside
partkey_datum_from_expr before calling ExecInitExpr.

10% is more than a "slight" improvement, I'd say! It's certainly got
to be worth avoiding the repeated calls to ExecInitExpr, whatever we
do about the memory contexts.

I've attached a patch which does just this. On benchmarking again with
this single change performance has improved 15% over master.

Also, out of curiosity, I also checked what this performed like before
the run-time pruning patch was committed (5c0675215). Taking the
average of the times below, it seems without this patch the
performance of this case has improved about 356% and about 410% with
this patch. So, I agree, it might be worth considering.

create table p (a int, value int) partition by hash (a);
select 'create table p'||x|| ' partition of p for values with (modulus
10, remainder '||x||');' from generate_series(0,9) x;
\gexec
create table t1 (a int);

insert into p select x,x from generate_Series(1,1000) x;
insert into t1 select x from generate_series(1,1000) x;

create index on p(a);

set enable_hashjoin = 0;
set enable_mergejoin = 0;
explain analyze select count(*) from t1 inner join p on t1.a=p.a;

-- Unpatched
Execution Time: 20413.975 ms
Execution Time: 20232.050 ms
Execution Time: 20229.116 ms

-- Patched
Execution Time: 17758.111 ms
Execution Time: 17645.151 ms
Execution Time: 17492.260 ms

-- 5c0675215e153ba1297fd494b34af2fdebd645d1
Execution Time: 72875.161 ms
Execution Time: 71817.757 ms
Execution Time: 72411.730 ms

That's neat! Definitely agree that we should call ExecInitExpr just once
here. The patch looks good too, except the long line. Maybe:

@@ -1514,13 +1514,15 @@ ExecSetupPartitionPruneState(PlanState *planstate,
List *partitionpruneinfo)
             foreach(lc3, step->exprs)
             {
                 Expr       *expr = (Expr *) lfirst(lc3);
+                int         step_id = step->step.step_id;
                 /*
                  * partkey_datum_from_expr does not need an expression state
                  * to evaluate a Const.
                  */
                 if (!IsA(expr, Const))
-                    context->exprstates[step->step.step_id * partnatts +
keyno] = ExecInitExpr(expr, context->planstate);
+                    context->exprstates[step_id * partnatts + keyno] =
+                                    ExecInitExpr(expr, context->planstate);

Thanks,
Amit

#160Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: David Rowley (#157)
Re: [HACKERS] Runtime Partition Pruning

Anybody wanna argue against pushing this patch now? I'm inclined to
push it on the grounds of being closure for already committed work, but
there are possible arguments about this being new development after
feature freeze.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#161Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Amit Langote (#159)
1 attachment(s)
Re: [HACKERS] Runtime Partition Pruning

Amit Langote wrote:

That's neat! Definitely agree that we should call ExecInitExpr just once
here. The patch looks good too, except the long line.

How about this as a small tweak? Determine the array index using a
macro, which serves as documentation.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

Attachments:

v3-0001-Initialize-ExprStates-once-in-run-time-partition-.patchtext/plain; charset=iso-8859-1Download
From 135740cd17e8c05f36a3b91b3af29515df6a2d3b Mon Sep 17 00:00:00 2001
From: "dgrowley@gmail.com" <dgrowley@gmail.com>
Date: Thu, 19 Apr 2018 11:51:16 +1200
Subject: [PATCH v3] Initialize ExprStates once in run-time partition pruning
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

Instead of doing ExecInitExpr every time a Param needs to be evaluated
in run-time partition pruning, do it once during run-time pruning
set-up and cache the exprstate in PartitionPruneContext, saving a lot of
work.

Author: David Rowley
Reviewed-by: Amit Langote, Álvaro Herrera
Discussion: https://postgr.es/m/CAKJS1f8-x+q-90QAPDu_okhQBV4DPEtPz8CJ=m0940GyT4DA4w@mail.gmail.com
---
 src/backend/executor/execPartition.c | 35 +++++++++++++++++++++++++++++++++++
 src/backend/partitioning/partprune.c | 27 +++++++++++++++++----------
 src/include/partitioning/partprune.h |  9 +++++++++
 3 files changed, 61 insertions(+), 10 deletions(-)

diff --git a/src/backend/executor/execPartition.c b/src/backend/executor/execPartition.c
index f7bbb804aa..f7418f64b1 100644
--- a/src/backend/executor/execPartition.c
+++ b/src/backend/executor/execPartition.c
@@ -1442,7 +1442,9 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		PartitionDesc partdesc;
 		Relation	rel;
 		PartitionKey partkey;
+		ListCell   *lc2;
 		int			partnatts;
+		int			n_steps;
 
 		pprune->present_parts = bms_copy(pinfo->present_parts);
 		pprune->subnode_map = palloc(sizeof(int) * pinfo->nparts);
@@ -1465,6 +1467,7 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 
 		partkey = RelationGetPartitionKey(rel);
 		partdesc = RelationGetPartitionDesc(rel);
+		n_steps = list_length(pinfo->pruning_steps);
 
 		context->strategy = partkey->strategy;
 		context->partnatts = partnatts = partkey->partnatts;
@@ -1476,6 +1479,38 @@ ExecSetupPartitionPruneState(PlanState *planstate, List *partitionpruneinfo)
 		context->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
 		context->planstate = planstate;
 		context->safeparams = NULL; /* empty for now */
+		context->exprstates = palloc0(sizeof(ExprState *) * n_steps * partnatts);
+
+		/* Initialize expression states for each expression */
+		foreach(lc2, pinfo->pruning_steps)
+		{
+			PartitionPruneStepOp *step = (PartitionPruneStepOp *) lfirst(lc2);
+			ListCell   *lc3;
+			int			keyno;
+
+			/* not needed for other step kinds */
+			if (!IsA(step, PartitionPruneStepOp))
+				continue;
+
+			Assert(list_length(step->exprs) <= partnatts);
+
+			keyno = 0;
+			foreach(lc3, step->exprs)
+			{
+				Expr	   *expr = (Expr *) lfirst(lc3);
+				int			stateidx;
+
+				/* not needed for Consts */
+				if (!IsA(expr, Const))
+				{
+					stateidx = PruneCxtStateIdx(partnatts,
+												step->step.step_id, keyno);
+					context->exprstates[stateidx] =
+						ExecInitExpr(expr, context->planstate);
+				}
+				keyno++;
+			}
+		}
 
 		pprune->pruning_steps = pinfo->pruning_steps;
 		pprune->extparams = bms_copy(pinfo->extparams);
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
index f8844ef2eb..62159477c1 100644
--- a/src/backend/partitioning/partprune.c
+++ b/src/backend/partitioning/partprune.c
@@ -169,7 +169,7 @@ static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *cont
 static bool match_boolean_partition_clause(Oid partopfamily, Expr *clause,
 							   Expr *partkey, Expr **outconst);
 static bool partkey_datum_from_expr(PartitionPruneContext *context,
-						Expr *expr, Datum *value);
+						Expr *expr, int stateidx, Datum *value);
 
 /*
  * make_partition_pruneinfo
@@ -444,6 +444,7 @@ prune_append_rel_partitions(RelOptInfo *rel)
 	/* Not valid when being called from the planner */
 	context.planstate = NULL;
 	context.safeparams = NULL;
+	context.exprstates = NULL;
 
 	/* Actual pruning happens here. */
 	partindexes = get_matching_partitions(&context, pruning_steps);
@@ -2788,10 +2789,13 @@ perform_pruning_base_step(PartitionPruneContext *context,
 		if (lc1 != NULL)
 		{
 			Expr	   *expr;
+			int			stateidx;
 			Datum		datum;
 
 			expr = lfirst(lc1);
-			if (partkey_datum_from_expr(context, expr, &datum))
+			stateidx = PruneCxtStateIdx(context->partnatts,
+										opstep->step.step_id, keyno);
+			if (partkey_datum_from_expr(context, expr, stateidx, &datum))
 			{
 				Oid			cmpfn;
 
@@ -3025,12 +3029,15 @@ match_boolean_partition_clause(Oid partopfamily, Expr *clause, Expr *partkey,
 
 /*
  * partkey_datum_from_expr
- *		Evaluate 'expr', set *value to the resulting Datum. Return true if
- *		evaluation was possible, otherwise false.
+ *		Evaluate expression for potential partition pruning
+ *
+ * Evaluate 'expr', whose ExprState is stateidx of the context exprstate
+ * array; set *value to the resulting Datum.  Return true if evaluation was
+ * possible, otherwise false.
  */
 static bool
 partkey_datum_from_expr(PartitionPruneContext *context,
-						Expr *expr, Datum *value)
+						Expr *expr, int stateidx, Datum *value)
 {
 	switch (nodeTag(expr))
 	{
@@ -3048,18 +3055,18 @@ partkey_datum_from_expr(PartitionPruneContext *context,
 				bms_is_member(((Param *) expr)->paramid, context->safeparams))
 			{
 				ExprState  *exprstate;
+				ExprContext *ectx;
 				bool		isNull;
 
-				exprstate = ExecInitExpr(expr, context->planstate);
-
-				*value = ExecEvalExprSwitchContext(exprstate,
-												   context->planstate->ps_ExprContext,
-												   &isNull);
+				exprstate = context->exprstates[stateidx];
+				ectx = context->planstate->ps_ExprContext;
+				*value = ExecEvalExprSwitchContext(exprstate, ectx, &isNull);
 				if (isNull)
 					return false;
 
 				return true;
 			}
+			break;
 
 		default:
 			break;
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
index a5568abce6..c9fe95dc30 100644
--- a/src/include/partitioning/partprune.h
+++ b/src/include/partitioning/partprune.h
@@ -50,8 +50,17 @@ typedef struct PartitionPruneContext
 	 * are not safe to use until the executor is running.
 	 */
 	Bitmapset  *safeparams;
+
+	/*
+	 * Array of ExprStates, indexed as per PruneCtxStateIdx; one for each
+	 * partkey in each pruning step.  Allocated if planstate is non-NULL,
+	 * otherwise NULL.
+	 */
+	ExprState **exprstates;
 } PartitionPruneContext;
 
+#define PruneCxtStateIdx(partnatts, step_id, keyno) \
+	((partnatts) * (step_id) + (keyno))
 
 extern List *make_partition_pruneinfo(PlannerInfo *root, List *partition_rels,
 						 List *subpaths, List *prunequal);
-- 
2.11.0

#162Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Alvaro Herrera (#161)
Re: [HACKERS] Runtime Partition Pruning

Pushed. Thanks!

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#163Andres Freund
andres@anarazel.de
In reply to: David Rowley (#157)
Re: [HACKERS] Runtime Partition Pruning

On 2018-04-19 12:04:35 +1200, David Rowley wrote:

On 19 April 2018 at 03:13, Robert Haas <robertmhaas@gmail.com> wrote:

10% is more than a "slight" improvement, I'd say! It's certainly got
to be worth avoiding the repeated calls to ExecInitExpr, whatever we
do about the memory contexts.

Yea, that seems important. Good that that got in.

What I wonder, after skimming this change, is where the relevant
expression context is reset? That's not really related to this change
but the wider thread, I just noticed it while looking at this.

Greetings,

Andres Freund

#164Alvaro Herrera
alvherre@alvh.no-ip.org
In reply to: Andres Freund (#163)
Re: [HACKERS] Runtime Partition Pruning

Andres Freund wrote:

What I wonder, after skimming this change, is where the relevant
expression context is reset? That's not really related to this change
but the wider thread, I just noticed it while looking at this.

Do you mean ResetExprContext? We use the plan's context, so it should
occur together with the plan reset itself, i.e. nodeAppend.c should do
it somewhere.

... Hmm, it appears we don't do it anywhere there actually.

--
�lvaro Herrera https://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#165David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#164)
Re: [HACKERS] Runtime Partition Pruning

On 25 April 2018 at 06:21, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Andres Freund wrote:

What I wonder, after skimming this change, is where the relevant
expression context is reset? That's not really related to this change
but the wider thread, I just noticed it while looking at this.

Do you mean ResetExprContext? We use the plan's context, so it should
occur together with the plan reset itself, i.e. nodeAppend.c should do
it somewhere.

... Hmm, it appears we don't do it anywhere there actually.

It's not immediately obvious to me why this is required.

All the expressions that are initialized here must live the entire
length of the executor run, and since they're allocated in the
ExecutorState context they'll be destroyed in FreeExecutorState().

If we do need to call ResetExprContext for these, then we'd just need
to invent a teardown for ExecSetupPartitionPruneState which would free
off the memory allocated and call ResetExprContext for all non-NULL
ExprStates in each context->exprstates. This function would need to be
called from the node's End function for any node that's set up a
PartitionPruneState.

Do we really need to do this given that the memory context these are
allocated in will be released a moment later anyway?

Just to ensure I'm not dreaming, I ran the following and couldn't see
the backend's memory consumption move.

create table lp (a int, value int) partition by list(a);
create table lp_1 partition of lp for values in(1);
create table lp_2 partition of lp for values in(2);
create function lp_value(p_a int) returns int as $$ select value from
lp where a = p_a; $$ language sql;
insert into lp values(1,10),(2,20);
select sum(lp_value(x)) from generate_Series(1,2) x,
generate_series(1,10000000);

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#166David Rowley
david.rowley@2ndquadrant.com
In reply to: Alvaro Herrera (#162)
Re: [HACKERS] Runtime Partition Pruning

On 25 April 2018 at 05:10, Alvaro Herrera <alvherre@alvh.no-ip.org> wrote:

Pushed. Thanks!

Thanks!

--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services

#167Peter Eisentraut
peter.eisentraut@2ndquadrant.com
In reply to: Alvaro Herrera (#144)
Re: [HACKERS] Runtime Partition Pruning

On 2018-04-10 23:32, Alvaro Herrera wrote:

To figure out, I used the attached patch (not intended for application)
to add a backtrace to each log message, plus a couple of accusatory
elog() calls in relation_open and ExecSetupPartitionPruneState.

What do people think about adding something like this errbacktrace()
from �lvaro's patch to core PostgreSQL? If we could devise a way to
selectively enable it, it might be an easier way for users to provide
backtraces from errors.

--
Peter Eisentraut http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#168Tom Lane
tgl@sss.pgh.pa.us
In reply to: Peter Eisentraut (#167)
Re: [HACKERS] Runtime Partition Pruning

Peter Eisentraut <peter.eisentraut@2ndquadrant.com> writes:

On 2018-04-10 23:32, Alvaro Herrera wrote:

To figure out, I used the attached patch (not intended for application)
to add a backtrace to each log message, plus a couple of accusatory
elog() calls in relation_open and ExecSetupPartitionPruneState.

What do people think about adding something like this errbacktrace()
from Álvaro's patch to core PostgreSQL? If we could devise a way to
selectively enable it, it might be an easier way for users to provide
backtraces from errors.

I think we did discuss it right after that, or somewhere nearby, and
concluded that the output is so imprecise that it's not really going
to be worth whatever portability issues we'd have to deal with.

I'd be all for a better version, but glibc's backtrace() just sucks,
at least given our coding style with lots of static functions.

regards, tom lane

#169Tom Lane
tgl@sss.pgh.pa.us
In reply to: Tom Lane (#168)
Re: [HACKERS] Runtime Partition Pruning

I wrote:

Peter Eisentraut <peter.eisentraut@2ndquadrant.com> writes:

What do people think about adding something like this errbacktrace()
from Álvaro's patch to core PostgreSQL?

I think we did discuss it right after that, or somewhere nearby, and
concluded that the output is so imprecise that it's not really going
to be worth whatever portability issues we'd have to deal with.

Hmm, after some digging in the archives, the closest thing I can find
is this thread:

/messages/by-id/CAMsr+YGL+yfWE=JvbUbnpWtrRZNey7hJ07+zT4bYJdVp4Szdrg@mail.gmail.com

where we discussed using libunwind instead, but people didn't like
the extra dependency.

However, I stand by the assertion that glibc's backtrace() is too
imprecise to be useful; I've experimented with it and despaired of
being able to tell where control had actually been.

regards, tom lane

#170Andres Freund
andres@anarazel.de
In reply to: Tom Lane (#169)
Re: [HACKERS] Runtime Partition Pruning

Hi,

On 2019-05-24 11:34:58 -0400, Tom Lane wrote:

I wrote:

Peter Eisentraut <peter.eisentraut@2ndquadrant.com> writes:

What do people think about adding something like this errbacktrace()
from �lvaro's patch to core PostgreSQL?

I think we did discuss it right after that, or somewhere nearby, and
concluded that the output is so imprecise that it's not really going
to be worth whatever portability issues we'd have to deal with.

Hmm, after some digging in the archives, the closest thing I can find
is this thread:

/messages/by-id/CAMsr+YGL+yfWE=JvbUbnpWtrRZNey7hJ07+zT4bYJdVp4Szdrg@mail.gmail.com

where we discussed using libunwind instead, but people didn't like
the extra dependency.

Hm, I didn't actually see that much concern about that. I still think we
should just go for libunwind. At least on debian it's likely to already
be installed:

andres@alap4:~$ apt rdepends libunwind8
libunwind8
Reverse Depends:
Depends: libunwind-dev (= 1.2.1-9)
Depends: linux-perf-4.16
Depends: linux-perf-4.15
Depends: linux-perf-4.14
Depends: rspamd
Depends: linux-perf-5.0
Depends: libjulia1
Depends: julia
Depends: geary
Depends: libunwind8-dbgsym (= 1.2.1-9)
Depends: xwayland
Depends: xvfb
Depends: xserver-xorg-core
Depends: xserver-xephyr
Depends: xnest
Depends: xdmx
Depends: trafficserver
Depends: tigervnc-standalone-server
Depends: tarantool
Depends: strace
Depends: spring
Depends: rspamd
Depends: linux-perf-4.19
Depends: libunwind-setjmp0
Depends: libeina1a
Depends: libjulia1
Depends: julia
Depends: intel-gpu-tools
Depends: libheaptrack
Depends: libgoogle-perftools4
Depends: libgoogle-glog0v5
Depends: gdnsd
Depends: libevas1-engines-x
Depends: libevas1

In particular strace, xserver-xorg-core, perf are reasonably likely to
already installed.

It's also not a large library. I'd bet if we made it an optional
build-time dependency it'd get included by just about every distro.

Greetings,

Andres Freund

#171Tom Lane
tgl@sss.pgh.pa.us
In reply to: Andres Freund (#170)
Re: [HACKERS] Runtime Partition Pruning

Andres Freund <andres@anarazel.de> writes:

On 2019-05-24 11:34:58 -0400, Tom Lane wrote:

Hmm, after some digging in the archives, the closest thing I can find
is this thread:
/messages/by-id/CAMsr+YGL+yfWE=JvbUbnpWtrRZNey7hJ07+zT4bYJdVp4Szdrg@mail.gmail.com
where we discussed using libunwind instead, but people didn't like
the extra dependency.

Hm, I didn't actually see that much concern about that. I still think we
should just go for libunwind.

Is it actually better? The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one. I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

regards, tom lane

#172Andres Freund
andres@anarazel.de
In reply to: Tom Lane (#171)
Re: [HACKERS] Runtime Partition Pruning

Hi,

On 2019-05-24 12:08:57 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

On 2019-05-24 11:34:58 -0400, Tom Lane wrote:

Hmm, after some digging in the archives, the closest thing I can find
is this thread:
/messages/by-id/CAMsr+YGL+yfWE=JvbUbnpWtrRZNey7hJ07+zT4bYJdVp4Szdrg@mail.gmail.com
where we discussed using libunwind instead, but people didn't like
the extra dependency.

Hm, I didn't actually see that much concern about that. I still think we
should just go for libunwind.

Is it actually better?

I've not looked in a while, but I think at some point it was.

The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one.

Does that depend on whether the program was compiled with
-fno-omit-frame-pointer? At least some distros now compile with frame
pointers enabled, to get reasonably fast perf profiles (at a basically
immeasurable slowdown, on modern-ish CPUs).

I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

It's still a lot easier to install debug symbols than to attach a
debugger and get a backtrace that way. Especially when the problem is
hard to reproduce.

Greetings,

Andres Freund

#173Tomas Vondra
tomas.vondra@2ndquadrant.com
In reply to: Andres Freund (#172)
Re: [HACKERS] Runtime Partition Pruning

On Fri, May 24, 2019 at 09:24:28AM -0700, Andres Freund wrote:

Hi,

On 2019-05-24 12:08:57 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:

On 2019-05-24 11:34:58 -0400, Tom Lane wrote:

Hmm, after some digging in the archives, the closest thing I can find
is this thread:
/messages/by-id/CAMsr+YGL+yfWE=JvbUbnpWtrRZNey7hJ07+zT4bYJdVp4Szdrg@mail.gmail.com
where we discussed using libunwind instead, but people didn't like
the extra dependency.

Hm, I didn't actually see that much concern about that. I still think we
should just go for libunwind.

Is it actually better?

I've not looked in a while, but I think at some point it was.

The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one.

Does that depend on whether the program was compiled with
-fno-omit-frame-pointer? At least some distros now compile with frame
pointers enabled, to get reasonably fast perf profiles (at a basically
immeasurable slowdown, on modern-ish CPUs).

I doubt that, because if that was the case we'd not be able to get
accurate profiles from perf, no? And AFAICS that's not the case,
irrespectedly of whether -fno-omit-frame-pointer is used.

I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

It's still a lot easier to install debug symbols than to attach a
debugger and get a backtrace that way. Especially when the problem is
hard to reproduce.

Right. Debugger requires interaction with a running process, while
having it integrated would make that unnecessary.

But I think Peter also suggested this might require the ability to
selectively enable the backtraces, and I think he's right. I doubt we
want to log them for every log message, right?

regards

--
Tomas Vondra http://www.2ndQuadrant.com
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services

#174Andres Freund
andres@anarazel.de
In reply to: Tomas Vondra (#173)
Re: [HACKERS] Runtime Partition Pruning

Hi,

On 2019-05-25 00:42:39 +0200, Tomas Vondra wrote:

On Fri, May 24, 2019 at 09:24:28AM -0700, Andres Freund wrote:

On 2019-05-24 12:08:57 -0400, Tom Lane wrote:

Andres Freund <andres@anarazel.de> writes:
The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one.

Does that depend on whether the program was compiled with
-fno-omit-frame-pointer? At least some distros now compile with frame
pointers enabled, to get reasonably fast perf profiles (at a basically
immeasurable slowdown, on modern-ish CPUs).

I doubt that, because if that was the case we'd not be able to get
accurate profiles from perf, no? And AFAICS that's not the case,
irrespectedly of whether -fno-omit-frame-pointer is used.

I can't parse this. With perf you can get accurate call-graph profiles
if you either use -fno-omit-frame-pointer, to force frame pointers to be
present (so the call graph can cheaply be assembled during profiling),
or with dwarf (the entire stack is saved, and then dwarf is unwinding at
perf report time - very large), or with lbr (CPU saves traces of
branches taken, enabling call graphs to be computed, but it needs they're not
very deep).

I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

It's still a lot easier to install debug symbols than to attach a
debugger and get a backtrace that way. Especially when the problem is
hard to reproduce.

Right. Debugger requires interaction with a running process, while
having it integrated would make that unnecessary.

But I think Peter also suggested this might require the ability to
selectively enable the backtraces, and I think he's right. I doubt we
want to log them for every log message, right?

Well, I think that if we had PANIC, SIGSEGV/BUS most FATALs covered,
we'd be off to a very good start. I'm not sure it's wise to give users
control over the computation of stack computations.

Greetings,

Andres Freund

#175Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#171)
Re: [HACKERS] Runtime Partition Pruning

On Fri, May 24, 2019 at 12:09 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Is it actually better? The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one. I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

I don't have an opinion on glibc vs. libunwind, but I don't understand
this argument. If you are unlucky enough to have a production server
that is crashing due to some hitherto-unknown bug, and if it's not
possible to get a good backtrace without installing debugging symbols,
then you are going to have to pick between (1) installing those
debugging symbols and (2) getting a poor backtrace. I don't really
see that as a problem so much as just the way life is. You can't
expect to get good debugging output without debugging symbols, just as
you can't expect to get a clean audit without bank statements or a
clear idea of how to find your way to an unknown destination without a
map. If you don't have the thing that contains the information that
is needed, you can't get the information; c'est la vie.

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#176Tom Lane
tgl@sss.pgh.pa.us
In reply to: Robert Haas (#175)
Re: [HACKERS] Runtime Partition Pruning

Robert Haas <robertmhaas@gmail.com> writes:

On Fri, May 24, 2019 at 12:09 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Is it actually better? The basic problem with backtrace() is that it
only knows about global functions, and so reports call sites in static
functions as if they were in whatever global function physically precedes
the static one. I think doing materially better requires depending on
debug symbols, which (at least in the Red Hat world) aren't going to
be there in a typical production situation.

I don't have an opinion on glibc vs. libunwind, but I don't understand
this argument. If you are unlucky enough to have a production server
that is crashing due to some hitherto-unknown bug, and if it's not
possible to get a good backtrace without installing debugging symbols,
then you are going to have to pick between (1) installing those
debugging symbols and (2) getting a poor backtrace. I don't really
see that as a problem so much as just the way life is.

Well, it *is* a problem. The whole point of this discussion I think is
to try to get better information "by default" for routine bug reports.
So if those come from production servers without debug symbols, which
I believe will be the usual case, then it seems likely to me that
libunwind will produce no better results than glibc. (But perhaps
I'm wrong about that --- I have not experimented with libunwind.)

Now it's true that "install debug symbols" is less of an ask than
"install debug symbols, *and* gdb, and make sure server core dumps are
enabled, and then go through this arcane manual procedure next time
you get a core dump". But we shouldn't fool ourselves that it isn't
an ask that's going to be hard for people with corporate policies
against installing extra stuff on production servers.

regards, tom lane

#177Robert Haas
robertmhaas@gmail.com
In reply to: Tom Lane (#176)
Re: [HACKERS] Runtime Partition Pruning

On Wed, May 29, 2019 at 6:02 PM Tom Lane <tgl@sss.pgh.pa.us> wrote:

Well, it *is* a problem. The whole point of this discussion I think is
to try to get better information "by default" for routine bug reports.
So if those come from production servers without debug symbols, which
I believe will be the usual case, then it seems likely to me that
libunwind will produce no better results than glibc. (But perhaps
I'm wrong about that --- I have not experimented with libunwind.)

Sure, I agree.

Now it's true that "install debug symbols" is less of an ask than
"install debug symbols, *and* gdb, and make sure server core dumps are
enabled, and then go through this arcane manual procedure next time
you get a core dump". But we shouldn't fool ourselves that it isn't
an ask that's going to be hard for people with corporate policies
against installing extra stuff on production servers.

There may be cases where that is true, but as you say, it's better
than what we have now. Plus, what exactly is the alternative? We
could:

- encourage packagers to install debug symbols by default (but they
might not; it might even be against policy), or
- invent our own system for generating backtraces and ignore what the
OS toolchain knows how to do (sounds painfully complex and expensive),
or
- just live with the fact that it's imperfect.

Is there a fourth option?

--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company

#178Andy Fan
zhihui.fan1213@gmail.com
In reply to: Robert Haas (#58)
Re: [HACKERS] Runtime Partition Pruning

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

I just realized this issue recently and reported it at [1]/messages/by-id/CA+HiwqGsP2L0BW1ad58HRSj1NouNSVHLfL5pm7=PBTvL0b+-BQ@mail.gmail.com, then Amit
pointed
me to this issue being discussed here, so I would like to continue this
topic
here.

I think we can split the issue into 2 issues. One is the partition prune
in initial
partition prune, which maybe happen in custom plan case only and caused
the above issue. The other one happens in the "Run-Time" partition prune,
I admit that is an important issue to resolve as well, but looks harder.
So I
think we can fix the first one at first.

The proposal to fix the first one is we can remember how many partitions
survived after plan time pruned for a RelOptInfo for a custom plan. and
record
such information in the CachedPlanSource. When we count for the cost of a
generic plan, we can reduce the cost based on such information.

Any thought about this? I'd be sorry if I missed some already existing
discussion
on this topic.

[1]: /messages/by-id/CA+HiwqGsP2L0BW1ad58HRSj1NouNSVHLfL5pm7=PBTvL0b+-BQ@mail.gmail.com
/messages/by-id/CA+HiwqGsP2L0BW1ad58HRSj1NouNSVHLfL5pm7=PBTvL0b+-BQ@mail.gmail.com

--
Best Regards
Andy Fan

#179Andy Fan
zhihui.fan1213@gmail.com
In reply to: Andy Fan (#178)
Re: [HACKERS] Runtime Partition Pruning

On Sun, Oct 4, 2020 at 3:10 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

I just realized this issue recently and reported it at [1], then Amit
pointed
me to this issue being discussed here, so I would like to continue this
topic
here.

I think we can split the issue into 2 issues. One is the partition prune
in initial
partition prune, which maybe happen in custom plan case only and caused
the above issue. The other one happens in the "Run-Time" partition prune,
I admit that is an important issue to resolve as well, but looks harder.
So I
think we can fix the first one at first.

... When we count for the cost of a
generic plan, we can reduce the cost based on such information.

This way doesn't work since after the initial partition prune, not only the
cost of the Append node should be reduced, the cost of other plans should
be reduced as well [1]/messages/by-id/CAKU4AWrWSCFO5fh01GTnN+1T8K8MyVAi4Gw-TvYC-Vhx3JohUw@mail.gmail.com

However I think if we can use partition prune information from a custom plan
at the cost_append_path stage, it looks the issue can be fixed. If so,
the idea
is similar to David's idea in [2]/messages/by-id/CAKJS1f8q_d7_Viweeivt1eS4Q8a0WAGFbrgeX38468mVgKseTA@mail.gmail.com, however Robert didn't agree with
this[2]/messages/by-id/CAKJS1f8q_d7_Viweeivt1eS4Q8a0WAGFbrgeX38468mVgKseTA@mail.gmail.com.
Can anyone elaborate this objection? for a partkey > $1 or BETWEEN cases,
some real results from the past are probably better than some hard-coded
assumptions IMO.

[1]: /messages/by-id/CAKU4AWrWSCFO5fh01GTnN+1T8K8MyVAi4Gw-TvYC-Vhx3JohUw@mail.gmail.com
/messages/by-id/CAKU4AWrWSCFO5fh01GTnN+1T8K8MyVAi4Gw-TvYC-Vhx3JohUw@mail.gmail.com

[2]: /messages/by-id/CAKJS1f8q_d7_Viweeivt1eS4Q8a0WAGFbrgeX38468mVgKseTA@mail.gmail.com
/messages/by-id/CAKJS1f8q_d7_Viweeivt1eS4Q8a0WAGFbrgeX38468mVgKseTA@mail.gmail.com

[3]: /messages/by-id/CA+TgmoZv8sd9cKyYtHwmd_13+BAjkVKo=ECe7G98tBK5Ejwatw@mail.gmail.com
/messages/by-id/CA+TgmoZv8sd9cKyYtHwmd_13+BAjkVKo=ECe7G98tBK5Ejwatw@mail.gmail.com

--
Best Regards
Andy Fan

#180Andy Fan
zhihui.fan1213@gmail.com
In reply to: Andy Fan (#179)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Oct 7, 2020 at 5:05 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

On Sun, Oct 4, 2020 at 3:10 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

I just realized this issue recently and reported it at [1], then Amit
pointed
me to this issue being discussed here, so I would like to continue this
topic
here.

I think we can split the issue into 2 issues. One is the partition prune
in initial
partition prune, which maybe happen in custom plan case only and caused
the above issue. The other one happens in the "Run-Time" partition
prune,
I admit that is an important issue to resolve as well, but looks harder.
So I
think we can fix the first one at first.

... When we count for the cost of a
generic plan, we can reduce the cost based on such information.

This way doesn't work since after the initial partition prune, not only
the
cost of the Append node should be reduced, the cost of other plans should
be reduced as well [1]

However I think if we can use partition prune information from a custom
plan
at the cost_append_path stage, it looks the issue can be fixed. If so,
the idea
is similar to David's idea in [2], however Robert didn't agree with
this[2].
Can anyone elaborate this objection? for a partkey > $1 or BETWEEN cases,
some real results from the past are probably better than some hard-coded
assumptions IMO.

I can understand Robert's idea now, he intended to resolve both the
"initial-partition-prune" case and "runtime partition prune" case while I
always think
about the former case (Amit reminded me about that at the beginning, but I
just
wake up right now..)

With the Robert's idea, I think we can do some hack at create_append_path,
we can figure out the pruneinfo (it was done at create_append_plan now) at
that
stage, and then did a fix_append_path with such pruneinfo. We need to fix
not
only the cost of AppendPath, but also rows of AppendPath, For example:
SELECT .. FROM t1, t2, p where t1.a = p.partkey and t1.b = t2.b, if the
plan is:
Nest Loop
Nest Loop
t1
Append (p)
t2

Then the rows of Append (p) will be very important.

For this idea, how to estimate how many partitions/rows can be pruned is a
key
part. Robert said "I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.". I think we can't say the 75% or 50% is a good
number, but the keypoint may be "partition_column = $1" is a common
case. And for the others case, we probably don't make it worse.

I think we need to do something here, any thoughts? Personally I'm more
like this idea now.

--
Best Regards
Andy Fan

#181Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Andy Fan (#180)
Re: [HACKERS] Runtime Partition Pruning

On Wed, Oct 7, 2020 at 7:00 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

On Wed, Oct 7, 2020 at 5:05 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

On Sun, Oct 4, 2020 at 3:10 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

I just realized this issue recently and reported it at [1], then Amit pointed
me to this issue being discussed here, so I would like to continue this topic
here.

I think we can split the issue into 2 issues. One is the partition prune in initial
partition prune, which maybe happen in custom plan case only and caused
the above issue. The other one happens in the "Run-Time" partition prune,
I admit that is an important issue to resolve as well, but looks harder. So I
think we can fix the first one at first.

... When we count for the cost of a
generic plan, we can reduce the cost based on such information.

This way doesn't work since after the initial partition prune, not only the
cost of the Append node should be reduced, the cost of other plans should
be reduced as well [1]

However I think if we can use partition prune information from a custom plan
at the cost_append_path stage, it looks the issue can be fixed. If so, the idea
is similar to David's idea in [2], however Robert didn't agree with this[2].
Can anyone elaborate this objection? for a partkey > $1 or BETWEEN cases,
some real results from the past are probably better than some hard-coded
assumptions IMO.

I can understand Robert's idea now, he intended to resolve both the
"initial-partition-prune" case and "runtime partition prune" case while I always think
about the former case (Amit reminded me about that at the beginning, but I just
wake up right now..)

With the Robert's idea, I think we can do some hack at create_append_path,
we can figure out the pruneinfo (it was done at create_append_plan now) at that
stage, and then did a fix_append_path with such pruneinfo. We need to fix not
only the cost of AppendPath, but also rows of AppendPath, For example:
SELECT .. FROM t1, t2, p where t1.a = p.partkey and t1.b = t2.b, if the
plan is:
Nest Loop
Nest Loop
t1
Append (p)
t2

Then the rows of Append (p) will be very important.

For this idea, how to estimate how many partitions/rows can be pruned is a key
part. Robert said "I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.". I think we can't say the 75% or 50% is a good
number, but the keypoint may be "partition_column = $1" is a common
case. And for the others case, we probably don't make it worse.

I think we need to do something here, any thoughts? Personally I'm more
like this idea now.

Yes, I think we have to do something on those lines. But I am
wondering why our stats machinery is failing to do that already. For
equality I think it's straight forward. But even for other operators
we should use the statistics from the partitioned table to estimate
the selectivity and then adjust number of scanned partitions = (number
of partitions * fraction of rows scanned). That might be better than
50% or 75%.

--
Best Wishes,
Ashutosh Bapat

#182Andy Fan
zhihui.fan1213@gmail.com
In reply to: Ashutosh Bapat (#181)
Re: [HACKERS] Runtime Partition Pruning

Hi Ashutosh:

On Thu, Oct 8, 2020 at 7:25 PM Ashutosh Bapat <ashutosh.bapat.oss@gmail.com>
wrote:

On Wed, Oct 7, 2020 at 7:00 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

On Wed, Oct 7, 2020 at 5:05 PM Andy Fan <zhihui.fan1213@gmail.com>

wrote:

On Sun, Oct 4, 2020 at 3:10 PM Andy Fan <zhihui.fan1213@gmail.com>

wrote:

Now, in my experience, the current system for custom plans vs. generic
plans doesn't approach the problem in this way at all, and in my
experience that results in some pretty terrible behavior. It will do
things like form a custom plan every time because the estimated cost
of the custom plan is lower than the estimated cost of the generic
plan even though the two plans are structurally identical; only the
estimates differ. It will waste gobs of CPU cycles by replanning a
primary key lookup 5 times just on the off chance that a lookup on the
primary key index isn't the best option. But this patch isn't going
to fix any of that. The best we can probably do is try to adjust the
costing for Append paths in some way that reflects the costs and
benefits of pruning. I'm tentatively in favor of trying to do
something modest in that area, but I don't have a detailed proposal.

I just realized this issue recently and reported it at [1], then Amit

pointed

me to this issue being discussed here, so I would like to continue

this topic

here.

I think we can split the issue into 2 issues. One is the partition

prune in initial

partition prune, which maybe happen in custom plan case only and caused
the above issue. The other one happens in the "Run-Time" partition

prune,

I admit that is an important issue to resolve as well, but looks

harder. So I

think we can fix the first one at first.

... When we count for the cost of a
generic plan, we can reduce the cost based on such information.

This way doesn't work since after the initial partition prune, not

only the

cost of the Append node should be reduced, the cost of other plans

should

be reduced as well [1]

However I think if we can use partition prune information from a custom

plan

at the cost_append_path stage, it looks the issue can be fixed. If

so, the idea

is similar to David's idea in [2], however Robert didn't agree with

this[2].

Can anyone elaborate this objection? for a partkey > $1 or BETWEEN

cases,

some real results from the past are probably better than some hard-coded
assumptions IMO.

I can understand Robert's idea now, he intended to resolve both the
"initial-partition-prune" case and "runtime partition prune" case while

I always think

about the former case (Amit reminded me about that at the beginning, but

I just

wake up right now..)

With the Robert's idea, I think we can do some hack at

create_append_path,

we can figure out the pruneinfo (it was done at create_append_plan now)

at that

stage, and then did a fix_append_path with such pruneinfo. We need to

fix not

only the cost of AppendPath, but also rows of AppendPath, For example:
SELECT .. FROM t1, t2, p where t1.a = p.partkey and t1.b = t2.b, if the
plan is:
Nest Loop
Nest Loop
t1
Append (p)
t2

Then the rows of Append (p) will be very important.

For this idea, how to estimate how many partitions/rows can be pruned

is a key

part. Robert said "I was thinking, rather, that if we know for example

that

we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.". I think we can't say the 75% or 50% is a good
number, but the keypoint may be "partition_column = $1" is a common
case. And for the others case, we probably don't make it worse.

I think we need to do something here, any thoughts? Personally I'm more
like this idea now.

Yes, I think we have to do something on those lines. But I am
wondering why our stats machinery is failing to do that already. For
equality I think it's straight forward. But even for other operators
we should use the statistics from the partitioned table to estimate
the selectivity and then adjust number of scanned partitions = (number
of partitions * fraction of rows scanned). That might be better than
50% or 75%.

Sorry for the late reply! Suppose we have partition defined like this:
p
- p1
- p2

When you talk about "the statistics from the partitioned table", do you
mean the statistics from p or p1/p2? I just confirmed there is no
statistics
for p (at least pg_class.reltuples = -1), so I think you are talking about
p1/p2.

Here we are talking about partkey = $1 or partkey = RunTimeValue.
so even the value can hit 1 partition only, but since we don't know
the exact value, so we treat all the partition equally. so looks
nothing wrong with partition level estimation. However when we cost
the Append path, we need know just one of them can be hit, then
we need do something there. Both AppendPath->rows/total_cost
should be adjusted (That doesn't happen now).

--
Best Regards
Andy Fan

#183Ashutosh Bapat
ashutosh.bapat.oss@gmail.com
In reply to: Andy Fan (#182)
Re: [HACKERS] Runtime Partition Pruning

On Mon, Oct 12, 2020 at 7:59 AM Andy Fan <zhihui.fan1213@gmail.com> wrote:

Sorry for the late reply! Suppose we have partition defined like this:
p
- p1
- p2

When you talk about "the statistics from the partitioned table", do you
mean the statistics from p or p1/p2? I just confirmed there is no statistics
for p (at least pg_class.reltuples = -1), so I think you are talking about
p1/p2.

I am talking about p when I say statistics from the partitioned table.
I see that pg_statistic row from p is well populated.
pg_class.reltuples = -1 indicates that the heap doesn't have any rows.
set_rel_size() sets the number of rows in the partitioned table based
on the rows in individual unpruned partitions.

Here we are talking about partkey = $1 or partkey = RunTimeValue.
so even the value can hit 1 partition only, but since we don't know
the exact value, so we treat all the partition equally. so looks
nothing wrong with partition level estimation. However when we cost
the Append path, we need know just one of them can be hit, then
we need do something there. Both AppendPath->rows/total_cost
should be adjusted (That doesn't happen now).

I think in this case we can safely assume that only one partition will
remain so normalize costs considering that only one partition will
survive.

--
Best Wishes,
Ashutosh Bapat

#184Andy Fan
zhihui.fan1213@gmail.com
In reply to: Ashutosh Bapat (#183)
Re: [HACKERS] Runtime Partition Pruning

On Mon, Oct 12, 2020 at 5:48 PM Ashutosh Bapat <ashutosh.bapat.oss@gmail.com>
wrote:

On Mon, Oct 12, 2020 at 7:59 AM Andy Fan <zhihui.fan1213@gmail.com> wrote:

Sorry for the late reply! Suppose we have partition defined like this:
p
- p1
- p2

When you talk about "the statistics from the partitioned table", do you
mean the statistics from p or p1/p2? I just confirmed there is no

statistics

for p (at least pg_class.reltuples = -1), so I think you are talking

about

p1/p2.

I am talking about p when I say statistics from the partitioned table.
I see that pg_statistic row from p is well populated.
pg_class.reltuples = -1 indicates that the heap doesn't have any rows.
set_rel_size() sets the number of rows in the partitioned table based
on the rows in individual unpruned partitions.

Glad to know that, Thanks for this info!

Here we are talking about partkey = $1 or partkey = RunTimeValue.
so even the value can hit 1 partition only, but since we don't know
the exact value, so we treat all the partition equally. so looks
nothing wrong with partition level estimation. However when we cost
the Append path, we need know just one of them can be hit, then
we need do something there. Both AppendPath->rows/total_cost
should be adjusted (That doesn't happen now).

I think in this case we can safely assume that only one partition will
remain so normalize costs considering that only one partition will
survive.

Exactly. What I am trying to do is fix this at create_append_path,
do you have different suggestions? about the pkey > $1 case, I think
even if we use the statistics from partition level, it would be
hard-code as well since we don't know what value $1 is.

I have gone through the main part of the RunTime partition prune, hope
I can update a runnable patch soon. The main idea is fix the rows/
costs at create_append_path stage. So any suggestion in a different
direction will be very useful.

--
Best Regards
Andy Fan

#185Amit Langote
amitlangote09@gmail.com
In reply to: Ashutosh Bapat (#181)
Re: [HACKERS] Runtime Partition Pruning

On Thu, Oct 8, 2020 at 8:25 PM Ashutosh Bapat
<ashutosh.bapat.oss@gmail.com> wrote:

On Wed, Oct 7, 2020 at 7:00 PM Andy Fan <zhihui.fan1213@gmail.com> wrote:

I can understand Robert's idea now, he intended to resolve both the
"initial-partition-prune" case and "runtime partition prune" case while I always think
about the former case (Amit reminded me about that at the beginning, but I just
wake up right now..)

With the Robert's idea, I think we can do some hack at create_append_path,
we can figure out the pruneinfo (it was done at create_append_plan now) at that
stage, and then did a fix_append_path with such pruneinfo. We need to fix not
only the cost of AppendPath, but also rows of AppendPath, For example:
SELECT .. FROM t1, t2, p where t1.a = p.partkey and t1.b = t2.b, if the
plan is:
Nest Loop
Nest Loop
t1
Append (p)
t2

Then the rows of Append (p) will be very important.

For this idea, how to estimate how many partitions/rows can be pruned is a key
part. Robert said "I was thinking, rather, that if we know for example that
we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.". I think we can't say the 75% or 50% is a good
number, but the keypoint may be "partition_column = $1" is a common
case. And for the others case, we probably don't make it worse.

I think we need to do something here, any thoughts? Personally I'm more
like this idea now.

Yes, I think we have to do something on those lines. But I am
wondering why our stats machinery is failing to do that already. For
equality I think it's straight forward. But even for other operators
we should use the statistics from the partitioned table to estimate
the selectivity and then adjust number of scanned partitions = (number
of partitions * fraction of rows scanned). That might be better than
50% or 75%.

This is an interesting idea, that is, the idea of consulting parent
relation's stats to guess at the number of partitions that might be
scanned.

However, we don't currently consult an inheritance parent relation's
stats at all during "base" relation planning, which is why you don't
see the parent relation's statistics reflected in the row count and
costs assigned to its (Append at al) paths. The hard-coded rule is to
sum up children's rows and their paths' costs; see cost_append().

My thinking on this was that we'd just extend that hard-coded rule to
take into account that run-time pruning, if applicable in a given
plan, will cause some of those child paths to be discarded. Maybe
Andy is headed in that direction?

--
Amit Langote
EDB: http://www.enterprisedb.com

#186Andy Fan
zhihui.fan1213@gmail.com
In reply to: Amit Langote (#185)
Re: [HACKERS] Runtime Partition Pruning

On Tue, Oct 13, 2020 at 3:48 PM Amit Langote <amitlangote09@gmail.com>
wrote:

On Thu, Oct 8, 2020 at 8:25 PM Ashutosh Bapat
<ashutosh.bapat.oss@gmail.com> wrote:

On Wed, Oct 7, 2020 at 7:00 PM Andy Fan <zhihui.fan1213@gmail.com>

wrote:

I can understand Robert's idea now, he intended to resolve both the
"initial-partition-prune" case and "runtime partition prune" case

while I always think

about the former case (Amit reminded me about that at the beginning,

but I just

wake up right now..)

With the Robert's idea, I think we can do some hack at

create_append_path,

we can figure out the pruneinfo (it was done at create_append_plan

now) at that

stage, and then did a fix_append_path with such pruneinfo. We need to

fix not

only the cost of AppendPath, but also rows of AppendPath, For example:
SELECT .. FROM t1, t2, p where t1.a = p.partkey and t1.b = t2.b, if the
plan is:
Nest Loop
Nest Loop
t1
Append (p)
t2

Then the rows of Append (p) will be very important.

For this idea, how to estimate how many partitions/rows can be pruned

is a key

part. Robert said "I was thinking, rather, that if we know for

example that

we've doing pruning on partition_column = $1, then we know that only
one partition will match. That's probably a common case. If we've
got partition_column > $1, we could assume that, say, 75% of the
partitions would match. partition_column BETWEEN $1 and $2 is
probably a bit more selective, so maybe we assume 50% of the
partitions would match.". I think we can't say the 75% or 50% is a

good

number, but the keypoint may be "partition_column = $1" is a common
case. And for the others case, we probably don't make it worse.

I think we need to do something here, any thoughts? Personally I'm more
like this idea now.

Yes, I think we have to do something on those lines. But I am
wondering why our stats machinery is failing to do that already. For
equality I think it's straight forward. But even for other operators
we should use the statistics from the partitioned table to estimate
the selectivity and then adjust number of scanned partitions = (number
of partitions * fraction of rows scanned). That might be better than
50% or 75%.

This is an interesting idea, that is, the idea of consulting parent
relation's stats to guess at the number of partitions that might be
scanned.

However, we don't currently consult an inheritance parent relation's
stats at all during "base" relation planning, which is why you don't
see the parent relation's statistics reflected in the row count and
costs assigned to its (Append at al) paths. The hard-coded rule is to
sum up children's rows and their paths' costs; see cost_append().

My thinking on this was that we'd just extend that hard-coded rule to
take into account that run-time pruning, if applicable in a given
plan, will cause some of those child paths to be discarded. Maybe
Andy is headed in that direction?

Yes, that's what I am trying to do. The minimum code in my mind is:

create_append_path(...)
{

double run_time_prune_est;

if (enable_partition_prune && .. )
List * partition_prune_step_ops = cal_prune_step_ops(rel,
partitioned_rels);
run_time_prune_est =
estimate_runtime_prune_ratio(partition_prune_step_ops);
}
// adjust the rows/costs of AppendPath based on the run_time_prune_est.

The difference between '=' and '<' will be handled in function
estimate_runtime_prune_ratio. Since I still don't make my mind runnable
now,
some data structures may change, but the overall idea is something like
above.

--
Best Regards
Andy Fan