#include "access/nbtree.h"
 #include "catalog/pg_operator.h"
 #include "catalog/pg_opfamily.h"
+#include "catalog/pg_proc.h"
 #include "catalog/pg_type.h"
 #include "executor/executor.h"
 #include "miscadmin.h"
  */
 typedef struct GeneratePruningStepsContext
 {
+       /* Input data: */
+       bool            forplanner;             /* true when generating steps to be used
+                                                                * during query planning */
+       /* Working state and result data: */
        int                     next_step_id;
-       List       *steps;
+       List       *steps;                      /* output, list of PartitionPruneSteps */
 } GeneratePruningStepsContext;
 
 /* The result of performing one PartitionPruneStep */
                                                          List *partitioned_rels, List *prunequal,
                                                          Bitmapset **matchedsubplans);
 static List *gen_partprune_steps(RelOptInfo *rel, List *clauses,
-                                       bool *contradictory);
+                                       bool forplanner, bool *contradictory);
 static List *gen_partprune_steps_internal(GeneratePruningStepsContext *context,
                                                         RelOptInfo *rel, List *clauses,
                                                         bool *contradictory);
                                                                                                  targetpart->relids);
                }
 
-               /* Convert pruning qual to pruning steps. */
-               pruning_steps = gen_partprune_steps(subpart, partprunequal,
+               /*
+                * Convert pruning qual to pruning steps.  Since these steps will be
+                * used in the executor, we can pass 'forplanner' as false to allow
+                * steps to be generated that are unsafe for evaluation during
+                * planning, e.g. evaluation of stable functions.
+                */
+               pruning_steps = gen_partprune_steps(subpart, partprunequal, false,
                                                                                        &contradictory);
 
                if (contradictory)
 /*
  * gen_partprune_steps
  *             Process 'clauses' (a rel's baserestrictinfo list of clauses) and return
- *             a list of "partition pruning steps"
+ *             a list of "partition pruning steps".
+ *
+ * 'forplanner' must be true when generating steps to be evaluated during
+ * query planning, false when generating steps to be used at run-time.
+ *
+ * The result generated with forplanner=false includes all clauses that
+ * are selected with forplanner=true, because in some cases we need a
+ * combination of clauses to prune successfully.  For example, if we
+ * are partitioning on a hash of columns A and B, and we have clauses
+ * "WHERE A=constant AND B=nonconstant", we can't do anything at plan
+ * time even though the first clause would be evaluable then.  And we
+ * must include the first clause when called with forplanner=false,
+ * or we'll fail to prune at run-time either.  This does mean that when
+ * called with forplanner=false, we may return steps that don't actually
+ * need to be executed at runtime; it's left to analyze_partkey_exprs()
+ * to (re)discover that.
  *
  * If the clauses in the input list are contradictory or there is a
- * pseudo-constant "false", *contradictory is set to true upon return.
+ * pseudo-constant "false", *contradictory is set to true upon return,
+ * else it's set false.
  */
 static List *
-gen_partprune_steps(RelOptInfo *rel, List *clauses, bool *contradictory)
+gen_partprune_steps(RelOptInfo *rel, List *clauses, bool forplanner,
+                                       bool *contradictory)
 {
        GeneratePruningStepsContext context;
 
+       context.forplanner = forplanner;
        context.next_step_id = 0;
        context.steps = NIL;
 
-       /* The clauses list may be modified below, so better make a copy. */
-       clauses = list_copy(clauses);
-
        /*
         * For sub-partitioned tables there's a corner case where if the
         * sub-partitioned table shares any partition keys with its parent, then
                if (rel->relid != 1)
                        ChangeVarNodes((Node *) partqual, 1, rel->relid, 0);
 
-               clauses = list_concat(clauses, partqual);
+               /* Use list_copy to avoid modifying the passed-in List */
+               clauses = list_concat(list_copy(clauses), partqual);
        }
 
        /* Down into the rabbit-hole. */
-       gen_partprune_steps_internal(&context, rel, clauses, contradictory);
+       (void) gen_partprune_steps_internal(&context, rel, clauses, contradictory);
 
        return context.steps;
 }
 
 /*
  * prune_append_rel_partitions
- *             Returns indexes into rel->part_rels of the minimum set of child
- *             partitions which must be scanned to satisfy rel's baserestrictinfo
- *             quals.
+ *             Process rel's baserestrictinfo and make use of quals which can be
+ *             evaluated during query planning in order to determine the minimum set
+ *             of partitions which must be scanned to satisfy these quals.  Returns
+ *             the matching partitions in the form of a Bitmapset containing the
+ *             partitions' indexes in the rel's part_rels array.
  *
  * Callers must ensure that 'rel' is a partitioned table.
  */
 
        /*
         * Process clauses.  If the clauses are found to be contradictory, we can
-        * return the empty set.
+        * return the empty set.  Pass 'forplanner' as true to indicate to the
+        * pruning code that we only want pruning steps that can be evaluated
+        * during planning.
         */
-       pruning_steps = gen_partprune_steps(rel, clauses, &contradictory);
+       pruning_steps = gen_partprune_steps(rel, clauses, true,
+                                                                               &contradictory);
        if (contradictory)
                return NULL;
 
  * get_matching_partitions
  *             Determine partitions that survive partition pruning
  *
+ * Note: context->planstate must be set to a valid PlanState when the
+ * pruning_steps were generated with 'forplanner' = false.
+ *
  * Returns a Bitmapset of the RelOptInfo->part_rels indexes of the surviving
  * partitions.
  */
  * clause that contains false, we set *contradictory to true and return NIL
  * (that is, no pruning steps).  Caller should consider all partitions as
  * pruned in that case.  Otherwise, *contradictory is set to false.
- *
- * Note: the 'clauses' List may be modified inside this function. Callers may
- * like to make a copy of it before passing them to this function.
  */
 static List *
 gen_partprune_steps_internal(GeneratePruningStepsContext *context,
        Expr       *expr;
 
        /*
-        * Recognize specially shaped clauses that match with the Boolean
-        * partition key.
+        * Recognize specially shaped clauses that match a Boolean partition key.
         */
        if (match_boolean_partition_clause(partopfamily, clause, partkey, &expr))
        {
                 * Matched with this key.  Now check various properties of the clause
                 * to see if it's sane to use it for pruning.  In most of these cases,
                 * we can return UNSUPPORTED because the same failure would occur no
-                * matter which partkey it's matched to.
+                * matter which partkey it's matched to.  (In particular, now that
+                * we've successfully matched one side of the opclause to a partkey,
+                * there is no chance that matching the other side to another partkey
+                * will produce a usable result, since that'd mean there are Vars on
+                * both sides.)
                 */
+               if (context->forplanner)
+               {
+                       /*
+                        * When pruning in the planner, we only support pruning using
+                        * comparisons to constants.  Immutable subexpressions will have
+                        * been folded to constants already, and we cannot prune on the
+                        * basis of anything that's not immutable.
+                        */
+                       if (!IsA(expr, Const))
+                               return PARTCLAUSE_UNSUPPORTED;
 
-               /*
-                * We can't prune using an expression with Vars.  (Report failure as
-                * UNSUPPORTED, not NOMATCH: as in the no-commutator case above, we
-                * now know there are Vars on both sides, so it's no good.)
-                */
-               if (contain_var_clause((Node *) expr))
-                       return PARTCLAUSE_UNSUPPORTED;
+                       /*
+                        * Also, the comparison operator itself must be immutable.
+                        */
+                       if (op_volatile(opno) != PROVOLATILE_IMMUTABLE)
+                               return PARTCLAUSE_UNSUPPORTED;
+               }
+               else
+               {
+                       /*
+                        * Otherwise, non-consts are allowed, but we can't prune using an
+                        * expression that contains Vars.
+                        */
+                       if (contain_var_clause((Node *) expr))
+                               return PARTCLAUSE_UNSUPPORTED;
+
+                       /*
+                        * And we must reject anything containing a volatile function.
+                        * Stable functions are OK though.  (We need not check this for
+                        * the comparison operator itself: anything that belongs to a
+                        * partitioning operator family must be at least stable.)
+                        */
+                       if (contain_volatile_functions((Node *) expr))
+                               return PARTCLAUSE_UNSUPPORTED;
+               }
 
                /*
                 * Only allow strict operators.  This will guarantee nulls are
                if (!op_strict(opno))
                        return PARTCLAUSE_UNSUPPORTED;
 
-               /* We can't use any volatile expressions to prune partitions. */
-               if (contain_volatile_functions((Node *) expr))
-                       return PARTCLAUSE_UNSUPPORTED;
-
                /*
                 * See if the operator is relevant to the partitioning opfamily.
                 *
                if (IsA(leftop, RelabelType))
                        leftop = ((RelabelType *) leftop)->arg;
 
-               /* Check it matches this partition key */
+               /* check if the LHS matches this partition key */
                if (!equal(leftop, partkey) ||
                        !PartCollMatchesExprColl(partcoll, saop->inputcollid))
                        return PARTCLAUSE_NOMATCH;
 
                /*
                 * Matched with this key.  Check various properties of the clause to
-                * see if it can sanely be used for partition pruning (this is mostly
-                * the same as for a plain OpExpr).
+                * see if it can sanely be used for partition pruning (this is
+                * identical to the logic for a plain OpExpr).
                 */
+               if (context->forplanner)
+               {
+                       /*
+                        * When pruning in the planner, we only support pruning using
+                        * comparisons to constants.  Immutable subexpressions will have
+                        * been folded to constants already, and we cannot prune on the
+                        * basis of anything that's not immutable.
+                        */
+                       if (!IsA(rightop, Const))
+                               return PARTCLAUSE_UNSUPPORTED;
 
-               /* We can't prune using an expression with Vars. */
-               if (contain_var_clause((Node *) rightop))
-                       return PARTCLAUSE_UNSUPPORTED;
+                       /*
+                        * Also, the comparison operator itself must be immutable.
+                        */
+                       if (op_volatile(saop_op) != PROVOLATILE_IMMUTABLE)
+                               return PARTCLAUSE_UNSUPPORTED;
+               }
+               else
+               {
+                       /*
+                        * Otherwise, non-consts are allowed, but we can't prune using an
+                        * expression that contains Vars.
+                        */
+                       if (contain_var_clause((Node *) rightop))
+                               return PARTCLAUSE_UNSUPPORTED;
+
+                       /*
+                        * And we must reject anything containing a volatile function.
+                        * Stable functions are OK though.  (We need not check this for
+                        * the comparison operator itself: anything that belongs to a
+                        * partitioning operator family must be at least stable.)
+                        */
+                       if (contain_volatile_functions((Node *) rightop))
+                               return PARTCLAUSE_UNSUPPORTED;
+               }
 
                /*
                 * Only allow strict operators.  This will guarantee nulls are
                if (!op_strict(saop_op))
                        return PARTCLAUSE_UNSUPPORTED;
 
-               /* We can't use any volatile expressions to prune partitions. */
-               if (contain_volatile_functions((Node *) rightop))
-                       return PARTCLAUSE_UNSUPPORTED;
-
                /*
                 * In case of NOT IN (..), we get a '<>', which we handle if list
                 * partitioning is in use and we're able to confirm that it's negator
 
        /*
         * Steps require run-time pruning if they contain EXEC_PARAM Params.
-        * Otherwise, if their expressions aren't simple Consts, they require
-        * startup-time pruning.
+        * Otherwise, if their expressions aren't simple Consts or they involve
+        * non-immutable comparison operators, they require startup-time pruning.
+        * (Otherwise, the pruning would have been done at plan time.)
+        *
+        * Notice that what we actually check for mutability is the comparison
+        * functions, not the original operators.  This relies on the support
+        * functions of the btree or hash opfamily being marked consistently with
+        * the operators.
         */
        pinfo->nexprs = list_length(steps) * partnatts;
        pinfo->hasexecparam = (bool *) palloc0(sizeof(bool) * pinfo->nexprs);
        {
                PartitionPruneStepOp *step = (PartitionPruneStepOp *) lfirst(lc);
                ListCell   *lc2;
+               ListCell   *lc3;
                int                     keyno;
 
                if (!IsA(step, PartitionPruneStepOp))
                        continue;
 
                keyno = 0;
-               foreach(lc2, step->exprs)
+               Assert(list_length(step->exprs) == list_length(step->cmpfns));
+               forboth(lc2, step->exprs, lc3, step->cmpfns)
                {
                        Expr       *expr = lfirst(lc2);
+                       Oid                     fnoid = lfirst_oid(lc3);
 
                        if (!IsA(expr, Const))
                        {
 
                                doruntimeprune = true;
                        }
+                       else if (func_volatile(fnoid) != PROVOLATILE_IMMUTABLE)
+                       {
+                               /* No exec params here, but must do initial pruning */
+                               pinfo->do_initial_prune = true;
+                               doruntimeprune = true;
+                       }
                        keyno++;
                }
        }
        }
        else
        {
+               /*
+                * We should never see a non-Const in a step unless we're running in
+                * the executor.
+                */
+               Assert(context->planstate != NULL);
+
                /*
                 * When called from the executor we'll have a valid planstate so we
                 * may be able to evaluate an expression which could not be folded to
                 * must be careful here to evaluate expressions containing PARAM_EXEC
                 * Params only when told it's OK.
                 */
-               if (context->planstate &&
-                       (context->evalexecparams ||
-                        !context->exprhasexecparam[stateidx]))
+               if (context->evalexecparams || !context->exprhasexecparam[stateidx])
                {
                        ExprState  *exprstate;
                        ExprContext *ectx;
 
 (7 rows)
 
 drop table listp;
+--
+-- check that stable query clauses are only used in run-time pruning
+--
+create table stable_qual_pruning (a timestamp) partition by range (a);
+create table stable_qual_pruning1 partition of stable_qual_pruning
+  for values from ('2000-01-01') to ('2000-02-01');
+create table stable_qual_pruning2 partition of stable_qual_pruning
+  for values from ('2000-02-01') to ('2000-03-01');
+create table stable_qual_pruning3 partition of stable_qual_pruning
+  for values from ('3000-02-01') to ('3000-03-01');
+-- comparison against a stable value requires run-time pruning
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning where a < localtimestamp;
+                           QUERY PLAN                           
+----------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Removed: 1
+   ->  Seq Scan on stable_qual_pruning1 (actual rows=0 loops=1)
+         Filter: (a < LOCALTIMESTAMP)
+   ->  Seq Scan on stable_qual_pruning2 (actual rows=0 loops=1)
+         Filter: (a < LOCALTIMESTAMP)
+(6 rows)
+
+-- timestamp < timestamptz comparison is only stable, not immutable
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning where a < '2000-02-01'::timestamptz;
+                                   QUERY PLAN                                   
+--------------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Removed: 2
+   ->  Seq Scan on stable_qual_pruning1 (actual rows=0 loops=1)
+         Filter: (a < 'Tue Feb 01 00:00:00 2000 PST'::timestamp with time zone)
+(4 rows)
+
+-- check ScalarArrayOp cases
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning
+  where a = any(array['2010-02-01', '2020-01-01']::timestamp[]);
+           QUERY PLAN           
+--------------------------------
+ Result (actual rows=0 loops=1)
+   One-Time Filter: false
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning
+  where a = any(array['2000-02-01', '2010-01-01']::timestamp[]);
+                                                   QUERY PLAN                                                   
+----------------------------------------------------------------------------------------------------------------
+ Seq Scan on stable_qual_pruning2 (actual rows=0 loops=1)
+   Filter: (a = ANY ('{"Tue Feb 01 00:00:00 2000","Fri Jan 01 00:00:00 2010"}'::timestamp without time zone[]))
+(2 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning
+  where a = any(array['2000-02-01', localtimestamp]::timestamp[]);
+                                                 QUERY PLAN                                                 
+------------------------------------------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Removed: 2
+   ->  Seq Scan on stable_qual_pruning2 (actual rows=0 loops=1)
+         Filter: (a = ANY (ARRAY['Tue Feb 01 00:00:00 2000'::timestamp without time zone, LOCALTIMESTAMP]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning
+  where a = any(array['2010-02-01', '2020-01-01']::timestamptz[]);
+                                                        QUERY PLAN                                                         
+---------------------------------------------------------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Removed: 2
+   ->  Seq Scan on stable_qual_pruning1 (never executed)
+         Filter: (a = ANY ('{"Mon Feb 01 00:00:00 2010 PST","Wed Jan 01 00:00:00 2020 PST"}'::timestamp with time zone[]))
+(4 rows)
+
+explain (analyze, costs off, summary off, timing off)
+select * from stable_qual_pruning
+  where a = any(array['2000-02-01', '2010-01-01']::timestamptz[]);
+                                                        QUERY PLAN                                                         
+---------------------------------------------------------------------------------------------------------------------------
+ Append (actual rows=0 loops=1)
+   Subplans Removed: 2
+   ->  Seq Scan on stable_qual_pruning2 (actual rows=0 loops=1)
+         Filter: (a = ANY ('{"Tue Feb 01 00:00:00 2000 PST","Fri Jan 01 00:00:00 2010 PST"}'::timestamp with time zone[]))
+(4 rows)
+
+drop table stable_qual_pruning;
 -- Ensure runtime pruning works with initplans params with boolean types
 create table boolvalues (value bool not null);
 insert into boolvalues values('t'),('f');