Faster partition pruning
authorAlvaro Herrera
Fri, 6 Apr 2018 19:23:04 +0000 (16:23 -0300)
committerAlvaro Herrera
Fri, 6 Apr 2018 19:44:05 +0000 (16:44 -0300)
Add a new module backend/partitioning/partprune.c, implementing a more
sophisticated algorithm for partition pruning.  The new module uses each
partition's "boundinfo" for pruning instead of constraint exclusion,
based on an idea proposed by Robert Haas of a "pruning program": a list
of steps generated from the query quals which are run iteratively to
obtain a list of partitions that must be scanned in order to satisfy
those quals.

At present, this targets planner-time partition pruning, but there exist
further patches to apply partition pruning at execution time as well.

This commit also moves some definitions from include/catalog/partition.h
to a new file include/partitioning/partbounds.h, in an attempt to
rationalize partitioning related code.

Authors: Amit Langote, David Rowley, Dilip Kumar
Reviewers: Robert Haas, Kyotaro Horiguchi, Ashutosh Bapat, Jesper Pedersen.
Discussion: https://postgr.es/m/098b9c71-1915-1a2a-8d52-1a7a50ce79e8@lab.ntt.co.jp

27 files changed:
src/backend/Makefile
src/backend/catalog/partition.c
src/backend/nodes/copyfuncs.c
src/backend/nodes/equalfuncs.c
src/backend/nodes/nodeFuncs.c
src/backend/nodes/outfuncs.c
src/backend/nodes/readfuncs.c
src/backend/optimizer/path/allpaths.c
src/backend/optimizer/path/indxpath.c
src/backend/optimizer/plan/planner.c
src/backend/optimizer/prep/prepunion.c
src/backend/optimizer/util/plancat.c
src/backend/optimizer/util/relnode.c
src/backend/partitioning/Makefile [new file with mode: 0644]
src/backend/partitioning/partprune.c [new file with mode: 0644]
src/include/catalog/catversion.h
src/include/catalog/partition.h
src/include/catalog/pg_opfamily.h
src/include/nodes/nodes.h
src/include/nodes/primnodes.h
src/include/nodes/relation.h
src/include/optimizer/planner.h
src/include/partitioning/partbounds.h [new file with mode: 0644]
src/include/partitioning/partprune.h [new file with mode: 0644]
src/test/regress/expected/inherit.out
src/test/regress/expected/partition_prune.out
src/test/regress/sql/partition_prune.sql

index a4b6d1658cc33bbeefa6d2a83959528d3c34cb46..42a0748ade55b7ab4307cab3b611d41801863fe0 100644 (file)
@@ -18,7 +18,8 @@ top_builddir = ../..
 include $(top_builddir)/src/Makefile.global
 
 SUBDIRS = access bootstrap catalog parser commands executor foreign lib libpq \
-   main nodes optimizer port postmaster regex replication rewrite \
+   main nodes optimizer partitioning port postmaster \
+   regex replication rewrite \
    statistics storage tcop tsearch utils $(top_builddir)/src/timezone \
    jit
 
index 275c52c4c3a88cacfce90bf792b7777e0c997fa1..4b31ff0ead4dd8bd62b44450c5694c6b4216aaba 100644 (file)
@@ -41,6 +41,7 @@
 #include "optimizer/prep.h"
 #include "optimizer/var.h"
 #include "parser/parse_coerce.h"
+#include "partitioning/partbounds.h"
 #include "rewrite/rewriteManip.h"
 #include "storage/lmgr.h"
 #include "utils/array.h"
 #include "utils/ruleutils.h"
 #include "utils/syscache.h"
 
-/*
- * Information about bounds of a partitioned relation
- *
- * A list partition datum that is known to be NULL is never put into the
- * datums array. Instead, it is tracked using the null_index field.
- *
- * In the case of range partitioning, ndatums will typically be far less than
- * 2 * nparts, because a partition's upper bound and the next partition's lower
- * bound are the same in most common cases, and we only store one of them (the
- * upper bound).  In case of hash partitioning, ndatums will be same as the
- * number of partitions.
- *
- * For range and list partitioned tables, datums is an array of datum-tuples
- * with key->partnatts datums each.  For hash partitioned tables, it is an array
- * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
- * given partition.
- *
- * The datums in datums array are arranged in increasing order as defined by
- * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
- * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
- * respectively. For range and list partitions this simply means that the
- * datums in the datums array are arranged in increasing order as defined by
- * the partition key's operator classes and collations.
- *
- * In the case of list partitioning, the indexes array stores one entry for
- * every datum, which is the index of the partition that accepts a given datum.
- * In case of range partitioning, it stores one entry per distinct range
- * datum, which is the index of the partition for which a given datum
- * is an upper bound.  In the case of hash partitioning, the number of the
- * entries in the indexes array is same as the greatest modulus amongst all
- * partitions.  For a given partition key datum-tuple, the index of the
- * partition which would accept that datum-tuple would be given by the entry
- * pointed by remainder produced when hash value of the datum-tuple is divided
- * by the greatest modulus.
- */
-
-typedef struct PartitionBoundInfoData
-{
-   char        strategy;       /* hash, list or range? */
-   int         ndatums;        /* Length of the datums following array */
-   Datum     **datums;
-   PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
-                                    * NULL for hash and list partitioned
-                                    * tables */
-   int        *indexes;        /* Partition indexes */
-   int         null_index;     /* Index of the null-accepting partition; -1
-                                * if there isn't one */
-   int         default_index;  /* Index of the default partition; -1 if there
-                                * isn't one */
-} PartitionBoundInfoData;
-
-#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
-#define partition_bound_has_default(bi) ((bi)->default_index != -1)
-
-/*
- * When qsort'ing partition bounds after reading from the catalog, each bound
- * is represented with one of the following structs.
- */
-
-/* One bound of a hash partition */
-typedef struct PartitionHashBound
-{
-   int         modulus;
-   int         remainder;
-   int         index;
-} PartitionHashBound;
-
-/* One value coming from some (index'th) list partition */
-typedef struct PartitionListValue
-{
-   int         index;
-   Datum       value;
-} PartitionListValue;
-
-/* One bound of a range partition */
-typedef struct PartitionRangeBound
-{
-   int         index;
-   Datum      *datums;         /* range bound datums */
-   PartitionRangeDatumKind *kind;  /* the kind of each datum */
-   bool        lower;          /* this is the lower (vs upper) bound */
-} PartitionRangeBound;
-
 
 static Oid get_partition_parent_worker(Relation inhRel, Oid relid);
 static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
@@ -173,29 +91,9 @@ static int32 partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc,
                     Oid *partcollation, Datum *datums1,
                     PartitionRangeDatumKind *kind1, bool lower1,
                     PartitionRangeBound *b2);
-static int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
-                          Oid *partcollation,
-                          Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
-                          Datum *tuple_datums, int n_tuple_datums);
-
-static int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
-                      PartitionBoundInfo boundinfo,
-                      Datum value, bool *is_equal);
-static int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
-                       Oid *partcollation,
-                       PartitionBoundInfo boundinfo,
-                       PartitionRangeBound *probe, bool *is_equal);
-static int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
-                             Oid *partcollation,
-                             PartitionBoundInfo boundinfo,
-                             int nvalues, Datum *values, bool *is_equal);
-static int partition_hash_bsearch(PartitionBoundInfo boundinfo,
-                      int modulus, int remainder);
 
 static int get_partition_bound_num_indexes(PartitionBoundInfo b);
-static int get_greatest_modulus(PartitionBoundInfo b);
-static uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
-                                Datum *values, bool *isnull);
+
 
 /*
  * RelationBuildPartitionDesc
@@ -765,13 +663,13 @@ partition_bounds_equal(int partnatts, int16 *parttyplen, bool *parttypbyval,
 
    if (b1->strategy == PARTITION_STRATEGY_HASH)
    {
-       int         greatest_modulus = get_greatest_modulus(b1);
+       int         greatest_modulus = get_hash_partition_greatest_modulus(b1);
 
        /*
         * If two hash partitioned tables have different greatest moduli,
         * their partition schemes don't match.
         */
-       if (greatest_modulus != get_greatest_modulus(b2))
+       if (greatest_modulus != get_hash_partition_greatest_modulus(b2))
            return false;
 
        /*
@@ -1029,7 +927,7 @@ check_new_partition_bound(char *relname, Relation parent,
                                (errcode(ERRCODE_INVALID_OBJECT_DEFINITION),
                                 errmsg("every hash partition modulus must be a factor of the next larger modulus")));
 
-                   greatest_modulus = get_greatest_modulus(boundinfo);
+                   greatest_modulus = get_hash_partition_greatest_modulus(boundinfo);
                    remainder = spec->remainder;
 
                    /*
@@ -1620,7 +1518,6 @@ get_partition_qual_relid(Oid relid)
    return result;
 }
 
-/* Module-local functions */
 
 /*
  * get_partition_operator
@@ -2637,7 +2534,7 @@ get_partition_for_tuple(Relation relation, Datum *values, bool *isnull)
        case PARTITION_STRATEGY_HASH:
            {
                PartitionBoundInfo boundinfo = partdesc->boundinfo;
-               int         greatest_modulus = get_greatest_modulus(boundinfo);
+               int         greatest_modulus = get_hash_partition_greatest_modulus(boundinfo);
                uint64      rowHash = compute_hash_value(key->partnatts,
                                                         key->partsupfunc,
                                                         values, isnull);
@@ -2971,7 +2868,7 @@ partition_rbound_cmp(int partnatts, FmgrInfo *partsupfunc, Oid *partcollation,
  * of attributes resp.
  *
  */
-static int32
+int32
 partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
                           Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
                           Datum *tuple_datums, int n_tuple_datums)
@@ -3005,7 +2902,7 @@ partition_rbound_datum_cmp(FmgrInfo *partsupfunc, Oid *partcollation,
  * *is_equal is set to true if the bound datum at the returned index is equal
  * to the input value.
  */
-static int
+int
 partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
                       PartitionBoundInfo boundinfo,
                       Datum value, bool *is_equal)
@@ -3048,7 +2945,7 @@ partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
  * *is_equal is set to true if the range bound at the returned index is equal
  * to the input range bound
  */
-static int
+int
 partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
                        Oid *partcollation,
                        PartitionBoundInfo boundinfo,
@@ -3093,7 +2990,7 @@ partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
  * *is_equal is set to true if the range bound at the returned index is equal
  * to the input tuple.
  */
-static int
+int
 partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
                              PartitionBoundInfo boundinfo,
                              int nvalues, Datum *values, bool *is_equal)
@@ -3136,7 +3033,7 @@ partition_range_datum_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
  *     less than or equal to the given (modulus, remainder) pair or -1 if
  *     all of them are greater
  */
-static int
+int
 partition_hash_bsearch(PartitionBoundInfo boundinfo,
                       int modulus, int remainder)
 {
@@ -3294,7 +3191,7 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
             * The number of the entries in the indexes array is same as the
             * greatest modulus.
             */
-           num_indexes = get_greatest_modulus(bound);
+           num_indexes = get_hash_partition_greatest_modulus(bound);
            break;
 
        case PARTITION_STRATEGY_LIST:
@@ -3315,14 +3212,14 @@ get_partition_bound_num_indexes(PartitionBoundInfo bound)
 }
 
 /*
- * get_greatest_modulus
+ * get_hash_partition_greatest_modulus
  *
  * Returns the greatest modulus of the hash partition bound. The greatest
  * modulus will be at the end of the datums array because hash partitions are
  * arranged in the ascending order of their modulus and remainders.
  */
-static int
-get_greatest_modulus(PartitionBoundInfo bound)
+int
+get_hash_partition_greatest_modulus(PartitionBoundInfo bound)
 {
    Assert(bound && bound->strategy == PARTITION_STRATEGY_HASH);
    Assert(bound->datums && bound->ndatums > 0);
@@ -3336,7 +3233,7 @@ get_greatest_modulus(PartitionBoundInfo bound)
  *
  * Compute the hash value for given not null partition key values.
  */
-static uint64
+uint64
 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
                   Datum *values, bool *isnull)
 {
index d2e4aa3c2f4eb8a66f31d24f582b8c857e581567..9287baaedc096f6d2ef67b4846dff803405ac7f6 100644 (file)
@@ -2150,6 +2150,38 @@ _copyMergeAction(const MergeAction *from)
    return newnode;
 }
 
+/*
+ * _copyPartitionPruneStepOp
+ */
+static PartitionPruneStepOp *
+_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);
+   COPY_BITMAPSET_FIELD(nullkeys);
+
+   return newnode;
+}
+
+/*
+ * _copyPartitionPruneStepCombine
+ */
+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);
+
+   return newnode;
+}
+
 /* ****************************************************************
  *                     relation.h copy functions
  *
@@ -2277,21 +2309,6 @@ _copyAppendRelInfo(const AppendRelInfo *from)
    return newnode;
 }
 
-/*
- * _copyPartitionedChildRelInfo
- */
-static PartitionedChildRelInfo *
-_copyPartitionedChildRelInfo(const PartitionedChildRelInfo *from)
-{
-   PartitionedChildRelInfo *newnode = makeNode(PartitionedChildRelInfo);
-
-   COPY_SCALAR_FIELD(parent_relid);
-   COPY_NODE_FIELD(child_rels);
-   COPY_SCALAR_FIELD(part_cols_updated);
-
-   return newnode;
-}
-
 /*
  * _copyPlaceHolderInfo
  */
@@ -5076,6 +5093,12 @@ copyObjectImpl(const void *from)
        case T_MergeAction:
            retval = _copyMergeAction(from);
            break;
+       case T_PartitionPruneStepOp:
+           retval = _copyPartitionPruneStepOp(from);
+           break;
+       case T_PartitionPruneStepCombine:
+           retval = _copyPartitionPruneStepCombine(from);
+           break;
 
            /*
             * RELATION NODES
@@ -5095,9 +5118,6 @@ copyObjectImpl(const void *from)
        case T_AppendRelInfo:
            retval = _copyAppendRelInfo(from);
            break;
-       case T_PartitionedChildRelInfo:
-           retval = _copyPartitionedChildRelInfo(from);
-           break;
        case T_PlaceHolderInfo:
            retval = _copyPlaceHolderInfo(from);
            break;
index f2dd9035df5d5b79ec0e4e91b5fdb5e490084544..d758515cfd10034748bd236f90871daf0b86a3d2 100644 (file)
@@ -915,16 +915,6 @@ _equalAppendRelInfo(const AppendRelInfo *a, const AppendRelInfo *b)
    return true;
 }
 
-static bool
-_equalPartitionedChildRelInfo(const PartitionedChildRelInfo *a, const PartitionedChildRelInfo *b)
-{
-   COMPARE_SCALAR_FIELD(parent_relid);
-   COMPARE_NODE_FIELD(child_rels);
-   COMPARE_SCALAR_FIELD(part_cols_updated);
-
-   return true;
-}
-
 static bool
 _equalPlaceHolderInfo(const PlaceHolderInfo *a, const PlaceHolderInfo *b)
 {
@@ -3230,9 +3220,6 @@ equal(const void *a, const void *b)
        case T_AppendRelInfo:
            retval = _equalAppendRelInfo(a, b);
            break;
-       case T_PartitionedChildRelInfo:
-           retval = _equalPartitionedChildRelInfo(a, b);
-           break;
        case T_PlaceHolderInfo:
            retval = _equalPlaceHolderInfo(a, b);
            break;
index f2f8227eb2ab15dc90386eddc789e5bdee4195b1..51c418778a03980f1fd0f9ffd5d5749e708569a7 100644 (file)
@@ -2156,6 +2156,17 @@ expression_tree_walker(Node *node,
                    return true;
            }
            break;
+       case T_PartitionPruneStepOp:
+           {
+               PartitionPruneStepOp *opstep = (PartitionPruneStepOp *) node;
+
+               if (walker((Node *) opstep->exprs, context))
+                   return true;
+           }
+           break;
+       case T_PartitionPruneStepCombine:
+           /* no expression subnodes */
+           break;
        case T_JoinExpr:
            {
                JoinExpr   *join = (JoinExpr *) node;
@@ -2958,6 +2969,20 @@ expression_tree_mutator(Node *node,
                return (Node *) newnode;
            }
            break;
+       case T_PartitionPruneStepOp:
+           {
+               PartitionPruneStepOp *opstep = (PartitionPruneStepOp *) node;
+               PartitionPruneStepOp *newnode;
+
+               FLATCOPY(newnode, opstep, PartitionPruneStepOp);
+               MUTATE(newnode->exprs, opstep->exprs, List *);
+
+               return (Node *) newnode;
+           }
+           break;
+       case T_PartitionPruneStepCombine:
+           /* no expression sub-nodes */
+           return (Node *) copyObject(node);
        case T_JoinExpr:
            {
                JoinExpr   *join = (JoinExpr *) node;
index a6a1c16164b3bde23214a9e55e79c85d8023f99c..03a91c3352da708db557eae0c07b0704eecce36a 100644 (file)
@@ -1710,6 +1710,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)
 {
@@ -2261,7 +2283,6 @@ _outPlannerInfo(StringInfo str, const PlannerInfo *node)
    WRITE_NODE_FIELD(full_join_clauses);
    WRITE_NODE_FIELD(join_info_list);
    WRITE_NODE_FIELD(append_rel_list);
-   WRITE_NODE_FIELD(pcinfo_list);
    WRITE_NODE_FIELD(rowMarks);
    WRITE_NODE_FIELD(placeholder_list);
    WRITE_NODE_FIELD(fkey_list);
@@ -2286,6 +2307,7 @@ _outPlannerInfo(StringInfo str, const PlannerInfo *node)
    WRITE_INT_FIELD(wt_param_id);
    WRITE_BITMAPSET_FIELD(curOuterRels);
    WRITE_NODE_FIELD(curOuterParams);
+   WRITE_BOOL_FIELD(partColsUpdated);
 }
 
 static void
@@ -2335,6 +2357,7 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node)
    WRITE_NODE_FIELD(joininfo);
    WRITE_BOOL_FIELD(has_eclass_joins);
    WRITE_BITMAPSET_FIELD(top_parent_relids);
+   WRITE_NODE_FIELD(partitioned_child_rels);
 }
 
 static void
@@ -2559,16 +2582,6 @@ _outAppendRelInfo(StringInfo str, const AppendRelInfo *node)
    WRITE_OID_FIELD(parent_reloid);
 }
 
-static void
-_outPartitionedChildRelInfo(StringInfo str, const PartitionedChildRelInfo *node)
-{
-   WRITE_NODE_TYPE("PARTITIONEDCHILDRELINFO");
-
-   WRITE_UINT_FIELD(parent_relid);
-   WRITE_NODE_FIELD(child_rels);
-   WRITE_BOOL_FIELD(part_cols_updated);
-}
-
 static void
 _outPlaceHolderInfo(StringInfo str, const PlaceHolderInfo *node)
 {
@@ -3973,6 +3986,12 @@ outNode(StringInfo str, const void *obj)
            case T_MergeAction:
                _outMergeAction(str, obj);
                break;
+           case T_PartitionPruneStepOp:
+               _outPartitionPruneStepOp(str, obj);
+               break;
+           case T_PartitionPruneStepCombine:
+               _outPartitionPruneStepCombine(str, obj);
+               break;
            case T_Path:
                _outPath(str, obj);
                break;
@@ -4114,9 +4133,6 @@ outNode(StringInfo str, const void *obj)
            case T_AppendRelInfo:
                _outAppendRelInfo(str, obj);
                break;
-           case T_PartitionedChildRelInfo:
-               _outPartitionedChildRelInfo(str, obj);
-               break;
            case T_PlaceHolderInfo:
                _outPlaceHolderInfo(str, obj);
                break;
index 37e35685956729c3bfb70e78c60c909e54d0c710..2812dc9646de6b37af1cb72af6746209f7b62e34 100644 (file)
@@ -1331,6 +1331,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();
+}
+
 /*
  * _readMergeAction
  */
@@ -2615,6 +2641,10 @@ parseNodeString(void)
        return_value = _readOnConflictExpr();
    else if (MATCH("MERGEACTION", 11))
        return_value = _readMergeAction();
+   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))
index c4e4db15a6822d1db05dd6dae89371ea539ebfee..65a34a255d2767910fbe567c575d3baa9b2c6a56 100644 (file)
@@ -43,6 +43,7 @@
 #include "optimizer/var.h"
 #include "parser/parse_clause.h"
 #include "parser/parsetree.h"
+#include "partitioning/partprune.h"
 #include "rewrite/rewriteManip.h"
 #include "utils/lsyscache.h"
 
@@ -874,12 +875,39 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
    double     *parent_attrsizes;
    int         nattrs;
    ListCell   *l;
+   Relids      live_children = NULL;
+   bool        did_pruning = false;
 
    /* Guard against stack overflow due to overly deep inheritance tree. */
    check_stack_depth();
 
    Assert(IS_SIMPLE_REL(rel));
 
+   /*
+    * Initialize partitioned_child_rels to contain this RT index.
+    *
+    * Note that during the set_append_rel_pathlist() phase, we will bubble up
+    * the indexes of partitioned relations that appear down in the tree, so
+    * that when we've created Paths for all the children, the root
+    * partitioned table's list will contain all such indexes.
+    */
+   if (rte->relkind == RELKIND_PARTITIONED_TABLE)
+       rel->partitioned_child_rels = list_make1_int(rti);
+
+   /*
+    * If the partitioned relation has any baserestrictinfo quals then we
+    * attempt to use these quals to prune away partitions that cannot
+    * possibly contain any tuples matching these quals.  In this case we'll
+    * store the relids of all partitions which could possibly contain a
+    * matching tuple, and skip anything else in the loop below.
+    */
+   if (rte->relkind == RELKIND_PARTITIONED_TABLE &&
+       rel->baserestrictinfo != NIL)
+   {
+       live_children = prune_append_rel_partitions(rel);
+       did_pruning = true;
+   }
+
    /*
     * Initialize to compute size estimates for whole append relation.
     *
@@ -1128,6 +1156,13 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel,
            continue;
        }
 
+       if (did_pruning && !bms_is_member(appinfo->child_relid, live_children))
+       {
+           /* This partition was pruned; skip it. */
+           set_dummy_rel_pathlist(childrel);
+           continue;
+       }
+
        if (relation_excluded_by_constraints(root, childrel, childRTE))
        {
            /*
@@ -1309,6 +1344,12 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
        if (IS_DUMMY_REL(childrel))
            continue;
 
+       /* Bubble up childrel's partitioned children. */
+       if (rel->part_scheme)
+           rel->partitioned_child_rels =
+               list_concat(rel->partitioned_child_rels,
+                           list_copy(childrel->partitioned_child_rels));
+
        /*
         * Child is live, so add it to the live_childrels list for use below.
         */
@@ -1346,49 +1387,55 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
    List       *all_child_outers = NIL;
    ListCell   *l;
    List       *partitioned_rels = NIL;
-   RangeTblEntry *rte;
    bool        build_partitioned_rels = false;
    double      partial_rows = -1;
 
-   if (IS_SIMPLE_REL(rel))
-   {
-       /*
-        * A root partition will already have a PartitionedChildRelInfo, and a
-        * non-root partitioned table doesn't need one, because its Append
-        * paths will get flattened into the parent anyway.  For a subquery
-        * RTE, no PartitionedChildRelInfo exists; we collect all
-        * partitioned_rels associated with any child.  (This assumes that we
-        * don't need to look through multiple levels of subquery RTEs; if we
-        * ever do, we could create a PartitionedChildRelInfo with the
-        * accumulated list of partitioned_rels which would then be found when
-        * populated our parent rel with paths.  For the present, that appears
-        * to be unnecessary.)
-        */
-       rte = planner_rt_fetch(rel->relid, root);
-       switch (rte->rtekind)
+   /*
+    * AppendPath generated for partitioned tables must record the RT indexes
+    * of partitioned tables that are direct or indirect children of this
+    * Append rel.
+    *
+    * AppendPath may be for a sub-query RTE (UNION ALL), in which case, 'rel'
+    * itself does not represent a partitioned relation, but the child sub-
+    * queries may contain references to partitioned relations.  The loop
+    * below will look for such children and collect them in a list to be
+    * passed to the path creation function.  (This assumes that we don't need
+    * to look through multiple levels of subquery RTEs; if we ever do, we
+    * could consider stuffing the list we generate here into sub-query RTE's
+    * RelOptInfo, just like we do for partitioned rels, which would be used
+    * when populating our parent rel with paths.  For the present, that
+    * appears to be unnecessary.)
+    */
+   if (rel->part_scheme != NULL)
+   {
+       if (IS_SIMPLE_REL(rel))
+           partitioned_rels = rel->partitioned_child_rels;
+       else if (IS_JOIN_REL(rel))
        {
-           case RTE_RELATION:
-               if (rte->relkind == RELKIND_PARTITIONED_TABLE)
-                   partitioned_rels =
-                       get_partitioned_child_rels(root, rel->relid, NULL);
-               break;
-           case RTE_SUBQUERY:
-               build_partitioned_rels = true;
-               break;
-           default:
-               elog(ERROR, "unexpected rtekind: %d", (int) rte->rtekind);
+           int         relid = -1;
+
+           /*
+            * For a partitioned joinrel, concatenate the component rels'
+            * partitioned_child_rels lists.
+            */
+           while ((relid = bms_next_member(rel->relids, relid)) >= 0)
+           {
+               RelOptInfo *component;
+
+               Assert(relid >= 1 && relid < root->simple_rel_array_size);
+               component = root->simple_rel_array[relid];
+               Assert(component->part_scheme != NULL);
+               Assert(list_length(component->partitioned_child_rels) >= 1);
+               partitioned_rels =
+                   list_concat(partitioned_rels,
+                               list_copy(component->partitioned_child_rels));
+           }
        }
+
+       Assert(list_length(partitioned_rels) >= 1);
    }
-   else if (rel->reloptkind == RELOPT_JOINREL && rel->part_scheme)
-   {
-       /*
-        * Associate PartitionedChildRelInfo of the root partitioned tables
-        * being joined with the root partitioned join (indicated by
-        * RELOPT_JOINREL).
-        */
-       partitioned_rels = get_partitioned_child_rels_for_join(root,
-                                                              rel->relids);
-   }
+   else if (rel->rtekind == RTE_SUBQUERY)
+       build_partitioned_rels = true;
 
    /*
     * For every non-dummy child, remember the cheapest path.  Also, identify
@@ -1407,9 +1454,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
         */
        if (build_partitioned_rels)
        {
-           List       *cprels;
+           List       *cprels = childrel->partitioned_child_rels;
 
-           cprels = get_partitioned_child_rels(root, childrel->relid, NULL);
            partitioned_rels = list_concat(partitioned_rels,
                                           list_copy(cprels));
        }
index 594ac8eacbea29c21564315bba1eaeac62a3038c..ec3f60d3115423f36544e3ac7a6720b1e642dd09 100644 (file)
@@ -40,9 +40,7 @@
 #include "utils/selfuncs.h"
 
 
-#define IsBooleanOpfamily(opfamily) \
-   ((opfamily) == BOOL_BTREE_FAM_OID || (opfamily) == BOOL_HASH_FAM_OID)
-
+/* XXX see PartCollMatchesExprColl */
 #define IndexCollMatchesExprColl(idxcollation, exprcollation) \
    ((idxcollation) == InvalidOid || (idxcollation) == (exprcollation))
 
index 15c8d34c704dd80486860fd90c4375e747508a8e..008492bad5e2e1a93926f7a486cebff1abe6b6c8 100644 (file)
@@ -616,7 +616,6 @@ subquery_planner(PlannerGlobal *glob, Query *parse,
    root->multiexpr_params = NIL;
    root->eq_classes = NIL;
    root->append_rel_list = NIL;
-   root->pcinfo_list = NIL;
    root->rowMarks = NIL;
    memset(root->upper_rels, 0, sizeof(root->upper_rels));
    memset(root->upper_targets, 0, sizeof(root->upper_targets));
@@ -631,6 +630,7 @@ subquery_planner(PlannerGlobal *glob, Query *parse,
    else
        root->wt_param_id = -1;
    root->non_recursive_path = NULL;
+   root->partColsUpdated = false;
 
    /*
     * If there is a WITH list, process each WITH query and build an initplan
@@ -1191,12 +1191,12 @@ inheritance_planner(PlannerInfo *root)
    ListCell   *lc;
    Index       rti;
    RangeTblEntry *parent_rte;
+   Relids      partitioned_relids = NULL;
    List       *partitioned_rels = NIL;
    PlannerInfo *parent_root;
    Query      *parent_parse;
    Bitmapset  *parent_relids = bms_make_singleton(top_parentRTindex);
    PlannerInfo **parent_roots = NULL;
-   bool        partColsUpdated = false;
 
    Assert(parse->commandType != CMD_INSERT);
 
@@ -1268,10 +1268,12 @@ inheritance_planner(PlannerInfo *root)
    if (parent_rte->relkind == RELKIND_PARTITIONED_TABLE)
    {
        nominalRelation = top_parentRTindex;
-       partitioned_rels = get_partitioned_child_rels(root, top_parentRTindex,
-                                                     &partColsUpdated);
-       /* The root partitioned table is included as a child rel */
-       Assert(list_length(partitioned_rels) >= 1);
+
+       /*
+        * Root parent's RT index is always present in the partitioned_rels of
+        * the ModifyTable node, if one is needed at all.
+        */
+       partitioned_relids = bms_make_singleton(top_parentRTindex);
    }
 
    /*
@@ -1502,6 +1504,15 @@ inheritance_planner(PlannerInfo *root)
        if (IS_DUMMY_PATH(subpath))
            continue;
 
+       /*
+        * Add the current parent's RT index to the partitione_rels set if
+        * we're going to create the ModifyTable path for a partitioned root
+        * table.
+        */
+       if (partitioned_relids)
+           partitioned_relids = bms_add_member(partitioned_relids,
+                                               appinfo->parent_relid);
+
        /*
         * If this is the first non-excluded child, its post-planning rtable
         * becomes the initial contents of final_rtable; otherwise, append
@@ -1603,6 +1614,21 @@ inheritance_planner(PlannerInfo *root)
    else
        rowMarks = root->rowMarks;
 
+   if (partitioned_relids)
+   {
+       int         i;
+
+       i = -1;
+       while ((i = bms_next_member(partitioned_relids, i)) >= 0)
+           partitioned_rels = lappend_int(partitioned_rels, i);
+
+       /*
+        * If we're going to create ModifyTable at all, the list should
+        * contain at least one member, that is, the root parent's index.
+        */
+       Assert(list_length(partitioned_rels) >= 1);
+   }
+
    /* Create Path representing a ModifyTable to do the UPDATE/DELETE work */
    add_path(final_rel, (Path *)
             create_modifytable_path(root, final_rel,
@@ -1610,7 +1636,7 @@ inheritance_planner(PlannerInfo *root)
                                     parse->canSetTag,
                                     nominalRelation,
                                     partitioned_rels,
-                                    partColsUpdated,
+                                    root->partColsUpdated,
                                     resultRelations,
                                     0,
                                     subpaths,
@@ -6144,65 +6170,6 @@ done:
    return parallel_workers;
 }
 
-/*
- * get_partitioned_child_rels
- *     Returns a list of the RT indexes of the partitioned child relations
- *     with rti as the root parent RT index. Also sets
- *     *part_cols_updated to true if any of the root rte's updated
- *     columns is used in the partition key either of the relation whose RTI
- *     is specified or of any child relation.
- *
- * Note: This function might get called even for range table entries that
- * are not partitioned tables; in such a case, it will simply return NIL.
- */
-List *
-get_partitioned_child_rels(PlannerInfo *root, Index rti,
-                          bool *part_cols_updated)
-{
-   List       *result = NIL;
-   ListCell   *l;
-
-   if (part_cols_updated)
-       *part_cols_updated = false;
-
-   foreach(l, root->pcinfo_list)
-   {
-       PartitionedChildRelInfo *pc = lfirst_node(PartitionedChildRelInfo, l);
-
-       if (pc->parent_relid == rti)
-       {
-           result = pc->child_rels;
-           if (part_cols_updated)
-               *part_cols_updated = pc->part_cols_updated;
-           break;
-       }
-   }
-
-   return result;
-}
-
-/*
- * get_partitioned_child_rels_for_join
- *     Build and return a list containing the RTI of every partitioned
- *     relation which is a child of some rel included in the join.
- */
-List *
-get_partitioned_child_rels_for_join(PlannerInfo *root, Relids join_relids)
-{
-   List       *result = NIL;
-   ListCell   *l;
-
-   foreach(l, root->pcinfo_list)
-   {
-       PartitionedChildRelInfo *pc = lfirst(l);
-
-       if (bms_is_member(pc->parent_relid, join_relids))
-           result = list_concat(result, list_copy(pc->child_rels));
-   }
-
-   return result;
-}
-
 /*
  * add_paths_to_grouping_rel
  *
index 5236ab378e14fd7f5aa528eca6c9cbdcdf5e2a84..67e47887fc88ef9717a5870d74cf06fcac5a4b68 100644 (file)
@@ -104,8 +104,7 @@ static void expand_partitioned_rtentry(PlannerInfo *root,
                           RangeTblEntry *parentrte,
                           Index parentRTindex, Relation parentrel,
                           PlanRowMark *top_parentrc, LOCKMODE lockmode,
-                          List **appinfos, List **partitioned_child_rels,
-                          bool *part_cols_updated);
+                          List **appinfos);
 static void expand_single_inheritance_child(PlannerInfo *root,
                                RangeTblEntry *parentrte,
                                Index parentRTindex, Relation parentrel,
@@ -1587,9 +1586,6 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
    /* Scan the inheritance set and expand it */
    if (RelationGetPartitionDesc(oldrelation) != NULL)
    {
-       List       *partitioned_child_rels = NIL;
-       bool        part_cols_updated = false;
-
        Assert(rte->relkind == RELKIND_PARTITIONED_TABLE);
 
        /*
@@ -1598,28 +1594,7 @@ expand_inherited_rtentry(PlannerInfo *root, RangeTblEntry *rte, Index rti)
         * extract the partition key columns of all the partitioned tables.
         */
        expand_partitioned_rtentry(root, rte, rti, oldrelation, oldrc,
-                                  lockmode, &root->append_rel_list,
-                                  &partitioned_child_rels,
-                                  &part_cols_updated);
-
-       /*
-        * We keep a list of objects in root, each of which maps a root
-        * partitioned parent RT index to the list of RT indexes of descendant
-        * partitioned child tables.  When creating an Append or a ModifyTable
-        * path for the parent, we copy the child RT index list verbatim to
-        * the path so that it could be carried over to the executor so that
-        * the latter could identify the partitioned child tables.
-        */
-       if (rte->inh && partitioned_child_rels != NIL)
-       {
-           PartitionedChildRelInfo *pcinfo;
-
-           pcinfo = makeNode(PartitionedChildRelInfo);
-           pcinfo->parent_relid = rti;
-           pcinfo->child_rels = partitioned_child_rels;
-           pcinfo->part_cols_updated = part_cols_updated;
-           root->pcinfo_list = lappend(root->pcinfo_list, pcinfo);
-       }
+                                  lockmode, &root->append_rel_list);
    }
    else
    {
@@ -1694,8 +1669,7 @@ static void
 expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
                           Index parentRTindex, Relation parentrel,
                           PlanRowMark *top_parentrc, LOCKMODE lockmode,
-                          List **appinfos, List **partitioned_child_rels,
-                          bool *part_cols_updated)
+                          List **appinfos)
 {
    int         i;
    RangeTblEntry *childrte;
@@ -1717,8 +1691,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
     * parentrte already has the root partrel's updatedCols translated to match
     * the attribute ordering of parentrel.
     */
-   if (!*part_cols_updated)
-       *part_cols_updated =
+   if (!root->partColsUpdated)
+       root->partColsUpdated =
            has_partition_attrs(parentrel, parentrte->updatedCols, NULL);
 
    /* First expand the partitioned table itself. */
@@ -1726,14 +1700,6 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
                                    top_parentrc, parentrel,
                                    appinfos, &childrte, &childRTindex);
 
-   /*
-    * The partitioned table does not have data for itself but still need to
-    * be locked. Update given list of partitioned children with RTI of this
-    * partitioned relation.
-    */
-   *partitioned_child_rels = lappend_int(*partitioned_child_rels,
-                                         childRTindex);
-
    for (i = 0; i < partdesc->nparts; i++)
    {
        Oid         childOID = partdesc->oids[i];
@@ -1760,8 +1726,7 @@ expand_partitioned_rtentry(PlannerInfo *root, RangeTblEntry *parentrte,
        if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
            expand_partitioned_rtentry(root, childrte, childRTindex,
                                       childrel, top_parentrc, lockmode,
-                                      appinfos, partitioned_child_rels,
-                                      part_cols_updated);
+                                      appinfos);
 
        /* Close child relation, but keep locks */
        heap_close(childrel, NoLock);
index 8a6baa7beae8cd448d1e8e19a66063d40325af85..52e4cca49aa25259f66607db58d010d15689ffed 100644 (file)
@@ -1171,7 +1171,6 @@ get_relation_constraints(PlannerInfo *root,
    Index       varno = rel->relid;
    Relation    relation;
    TupleConstr *constr;
-   List       *pcqual;
 
    /*
     * We assume the relation has already been safely locked.
@@ -1257,24 +1256,34 @@ get_relation_constraints(PlannerInfo *root,
        }
    }
 
-   /* Append partition predicates, if any */
-   pcqual = RelationGetPartitionQual(relation);
-   if (pcqual)
+   /*
+    * Append partition predicates, if any.
+    *
+    * For selects, partition pruning uses the parent table's partition bound
+    * descriptor, instead of constraint exclusion which is driven by the
+    * individual partition's partition constraint.
+    */
+   if (root->parse->commandType != CMD_SELECT)
    {
-       /*
-        * Run the partition quals through const-simplification similar to
-        * check constraints.  We skip canonicalize_qual, though, because
-        * partition quals should be in canonical form already; also, since
-        * the qual is in implicit-AND format, we'd have to explicitly convert
-        * it to explicit-AND format and back again.
-        */
-       pcqual = (List *) eval_const_expressions(root, (Node *) pcqual);
+       List       *pcqual = RelationGetPartitionQual(relation);
 
-       /* Fix Vars to have the desired varno */
-       if (varno != 1)
-           ChangeVarNodes((Node *) pcqual, 1, varno, 0);
+       if (pcqual)
+       {
+           /*
+            * Run the partition quals through const-simplification similar to
+            * check constraints.  We skip canonicalize_qual, though, because
+            * partition quals should be in canonical form already; also,
+            * since the qual is in implicit-AND format, we'd have to
+            * explicitly convert it to explicit-AND format and back again.
+            */
+           pcqual = (List *) eval_const_expressions(root, (Node *) pcqual);
 
-       result = list_concat(result, pcqual);
+           /* Fix Vars to have the desired varno */
+           if (varno != 1)
+               ChangeVarNodes((Node *) pcqual, 1, varno, 0);
+
+           result = list_concat(result, pcqual);
+       }
    }
 
    heap_close(relation, NoLock);
@@ -1869,6 +1878,7 @@ set_relation_partition_info(PlannerInfo *root, RelOptInfo *rel,
    rel->boundinfo = partition_bounds_copy(partdesc->boundinfo, partkey);
    rel->nparts = partdesc->nparts;
    set_baserel_partition_key_exprs(relation, rel);
+   rel->partition_qual = RelationGetPartitionQual(relation);
 }
 
 /*
@@ -1881,7 +1891,8 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
 {
    PartitionKey partkey = RelationGetPartitionKey(relation);
    ListCell   *lc;
-   int         partnatts;
+   int         partnatts,
+               i;
    PartitionScheme part_scheme;
 
    /* A partitioned table should have a partition key. */
@@ -1899,7 +1910,7 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
            partnatts != part_scheme->partnatts)
            continue;
 
-       /* Match the partition key types. */
+       /* Match partition key type properties. */
        if (memcmp(partkey->partopfamily, part_scheme->partopfamily,
                   sizeof(Oid) * partnatts) != 0 ||
            memcmp(partkey->partopcintype, part_scheme->partopcintype,
@@ -1917,6 +1928,19 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
        Assert(memcmp(partkey->parttypbyval, part_scheme->parttypbyval,
                      sizeof(bool) * partnatts) == 0);
 
+       /*
+        * If partopfamily and partopcintype matched, must have the same
+        * partition comparison functions.  Note that we cannot reliably
+        * Assert the equality of function structs themselves for they might
+        * be different across PartitionKey's, so just Assert for the function
+        * OIDs.
+        */
+#ifdef USE_ASSERT_CHECKING
+       for (i = 0; i < partkey->partnatts; i++)
+           Assert(partkey->partsupfunc[i].fn_oid ==
+                  part_scheme->partsupfunc[i].fn_oid);
+#endif
+
        /* Found matching partition scheme. */
        return part_scheme;
    }
@@ -1951,6 +1975,12 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
    memcpy(part_scheme->parttypbyval, partkey->parttypbyval,
           sizeof(bool) * partnatts);
 
+   part_scheme->partsupfunc = (FmgrInfo *)
+       palloc(sizeof(FmgrInfo) * partnatts);
+   for (i = 0; i < partnatts; i++)
+       fmgr_info_copy(&part_scheme->partsupfunc[i], &partkey->partsupfunc[i],
+                      CurrentMemoryContext);
+
    /* Add the partitioning scheme to PlannerInfo. */
    root->part_schemes = lappend(root->part_schemes, part_scheme);
 
index da8f0f93fc0d587f352c1d1633d5d8b5d3f62a69..b9aa7486bae8aa4122d8b05111152817b6f0710c 100644 (file)
@@ -154,9 +154,11 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent)
    rel->part_scheme = NULL;
    rel->nparts = 0;
    rel->boundinfo = NULL;
+   rel->partition_qual = NIL;
    rel->part_rels = NULL;
    rel->partexprs = NULL;
    rel->nullable_partexprs = NULL;
+   rel->partitioned_child_rels = NIL;
 
    /*
     * Pass top parent's relids down the inheritance hierarchy. If the parent
@@ -567,9 +569,11 @@ build_join_rel(PlannerInfo *root,
    joinrel->part_scheme = NULL;
    joinrel->nparts = 0;
    joinrel->boundinfo = NULL;
+   joinrel->partition_qual = NIL;
    joinrel->part_rels = NULL;
    joinrel->partexprs = NULL;
    joinrel->nullable_partexprs = NULL;
+   joinrel->partitioned_child_rels = NIL;
 
    /* Compute information relevant to the foreign relations. */
    set_foreign_rel_properties(joinrel, outer_rel, inner_rel);
@@ -734,9 +738,13 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel,
    joinrel->has_eclass_joins = false;
    joinrel->top_parent_relids = NULL;
    joinrel->part_scheme = NULL;
+   joinrel->nparts = 0;
+   joinrel->boundinfo = NULL;
+   joinrel->partition_qual = NIL;
    joinrel->part_rels = NULL;
    joinrel->partexprs = NULL;
    joinrel->nullable_partexprs = NULL;
+   joinrel->partitioned_child_rels = NIL;
 
    joinrel->top_parent_relids = bms_union(outer_rel->top_parent_relids,
                                           inner_rel->top_parent_relids);
diff --git a/src/backend/partitioning/Makefile b/src/backend/partitioning/Makefile
new file mode 100644 (file)
index 0000000..429207c
--- /dev/null
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+#    Makefile for backend/partitioning
+#
+# IDENTIFICATION
+#    src/backend/partitioning/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/partitioning
+top_builddir = ../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = partprune.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/partitioning/partprune.c b/src/backend/partitioning/partprune.c
new file mode 100644 (file)
index 0000000..959ee16
--- /dev/null
@@ -0,0 +1,2782 @@
+/*-------------------------------------------------------------------------
+ *
+ * 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.
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ *       src/backend/partitioning/partprune.c
+ *
+ *-------------------------------------------------------------------------
+*/
+#include "postgres.h"
+
+#include "access/hash.h"
+#include "access/nbtree.h"
+#include "catalog/pg_operator.h"
+#include "catalog/pg_opfamily.h"
+#include "catalog/pg_type.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/clauses.h"
+#include "optimizer/planner.h"
+#include "optimizer/predtest.h"
+#include "optimizer/prep.h"
+#include "partitioning/partprune.h"
+#include "partitioning/partbounds.h"
+#include "rewrite/rewriteManip.h"
+#include "utils/lsyscache.h"
+
+
+/*
+ * Information about a clause matched with a partition key.
+ */
+typedef struct PartClauseInfo
+{
+   int         keyno;          /* Partition key number (0 to partnatts - 1)  */
+   Oid         opno;           /* operator used to compare partkey to 'expr' */
+   bool        op_is_ne;       /* is clause's original operator <> ? */
+   Expr       *expr;           /* expr the partition key is compared to */
+   Oid         cmpfn;          /* Oid of function to compare 'expr' to the
+                                * partition key */
+   int         op_strategy;    /* cached info. */
+} PartClauseInfo;
+
+/*
+ * PartClauseMatchStatus
+ *     Describes the result match_clause_to_partition_key produces for a
+ *     given clause and the partition key to match with that are passed to it
+ */
+typedef enum PartClauseMatchStatus
+{
+   PARTCLAUSE_NOMATCH,
+   PARTCLAUSE_MATCH_CLAUSE,
+   PARTCLAUSE_MATCH_NULLNESS,
+   PARTCLAUSE_MATCH_STEPS,
+   PARTCLAUSE_MATCH_CONTRADICT,
+   PARTCLAUSE_UNSUPPORTED
+} PartClauseMatchStatus;
+
+/*
+ * GeneratePruningStepsContext
+ *     Information about the current state of generation of "pruning steps"
+ *     for a given set of clauses
+ *
+ * gen_partprune_steps() initializes an instance of this struct, which is used
+ * throughout the step generation process.
+ */
+typedef struct GeneratePruningStepsContext
+{
+   int         next_step_id;
+   List       *steps;
+} GeneratePruningStepsContext;
+
+/* The result of performing one PartitionPruneStep */
+typedef struct PruneStepResult
+{
+   /*
+    * The offsets of bounds (in a table's boundinfo) whose partition is
+    * selected by the pruning step.
+    */
+   Bitmapset  *bound_offsets;
+
+   bool        scan_default;   /* Scan the default partition? */
+   bool        scan_null;      /* Scan the partition for NULL values? */
+} PruneStepResult;
+
+
+static List *gen_partprune_steps_internal(GeneratePruningStepsContext *context,
+                            RelOptInfo *rel, List *clauses,
+                            bool *contradictory);
+static PartitionPruneStep *gen_prune_step_op(GeneratePruningStepsContext *context,
+                 StrategyNumber opstrategy, bool op_is_ne,
+                 List *exprs, List *cmpfns, Bitmapset *nullkeys);
+static PartitionPruneStep *gen_prune_step_combine(GeneratePruningStepsContext *context,
+                      List *source_stepids,
+                      PartitionPruneCombineOp combineOp);
+static PartitionPruneStep *gen_prune_steps_from_opexps(PartitionScheme part_scheme,
+                           GeneratePruningStepsContext *context,
+                           List **keyclauses, Bitmapset *nullkeys);
+static PartClauseMatchStatus match_clause_to_partition_key(RelOptInfo *rel,
+                             GeneratePruningStepsContext *context,
+                             Expr *clause, Expr *partkey, int partkeyidx,
+                             bool *clause_is_not_null,
+                             PartClauseInfo **pc, List **clause_steps);
+static List *get_steps_using_prefix(GeneratePruningStepsContext *context,
+                      StrategyNumber step_opstrategy,
+                      bool step_op_is_ne,
+                      Expr *step_lastexpr,
+                      Oid step_lastcmpfn,
+                      int step_lastkeyno,
+                      Bitmapset *step_nullkeys,
+                      List *prefix);
+static List *get_steps_using_prefix_recurse(GeneratePruningStepsContext *context,
+                              StrategyNumber step_opstrategy,
+                              bool step_op_is_ne,
+                              Expr *step_lastexpr,
+                              Oid step_lastcmpfn,
+                              int step_lastkeyno,
+                              Bitmapset *step_nullkeys,
+                              ListCell *start,
+                              List *step_exprs,
+                              List *step_cmpfns);
+static PruneStepResult *get_matching_hash_bounds(PartitionPruneContext *context,
+                        StrategyNumber opstrategy, Datum *values, int nvalues,
+                        FmgrInfo *partsupfunc, Bitmapset *nullkeys);
+static PruneStepResult *get_matching_list_bounds(PartitionPruneContext *context,
+                        StrategyNumber opstrategy, Datum value, int nvalues,
+                        FmgrInfo *partsupfunc, Bitmapset *nullkeys);
+static PruneStepResult *get_matching_range_bounds(PartitionPruneContext *context,
+                         StrategyNumber opstrategy, Datum *values, int nvalues,
+                         FmgrInfo *partsupfunc, Bitmapset *nullkeys);
+static PruneStepResult *perform_pruning_base_step(PartitionPruneContext *context,
+                         PartitionPruneStepOp *opstep);
+static PruneStepResult *perform_pruning_combine_step(PartitionPruneContext *context,
+                            PartitionPruneStepCombine *cstep,
+                            PruneStepResult **step_results);
+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);
+
+
+/*
+ * gen_partprune_steps
+ *     Process 'clauses' (a rel's baserestrictinfo list of clauses) and return
+ *     a list of "partition pruning steps"
+ *
+ * If the clauses in the input list are contradictory or there is a
+ * pseudo-constant "false", *contradictory is set to true upon return.
+ */
+List *
+gen_partprune_steps(RelOptInfo *rel, List *clauses, bool *contradictory)
+{
+   GeneratePruningStepsContext context;
+
+   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
+    * it's possible that the partitioning hierarchy allows the parent
+    * partition to only contain a narrower range of values than the
+    * sub-partitioned table does.  In this case it is possible that we'd
+    * include partitions that could not possibly have any tuples matching
+    * 'clauses'.  The possibility of such a partition arrangement is perhaps
+    * unlikely for non-default partitions, but it may be more likely in the
+    * case of default partitions, so we'll add the parent partition table's
+    * partition qual to the clause list in this case only.  This may result
+    * in the default partition being eliminated.
+    */
+   if (partition_bound_has_default(rel->boundinfo) &&
+       rel->partition_qual != NIL)
+   {
+       List       *partqual = rel->partition_qual;
+
+       partqual = (List *) expression_planner((Expr *) partqual);
+
+       /* Fix Vars to have the desired varno */
+       if (rel->relid != 1)
+           ChangeVarNodes((Node *) partqual, 1, rel->relid, 0);
+
+       clauses = list_concat(clauses, partqual);
+   }
+
+   /* Down into the rabbit-hole. */
+   gen_partprune_steps_internal(&context, rel, clauses, contradictory);
+
+   return context.steps;
+}
+
+/*
+ * prune_append_rel_partitions
+ *     Returns RT indexes of the minimum set of child partitions which must
+ *     be scanned to satisfy rel's baserestrictinfo quals.
+ *
+ * Callers must ensure that 'rel' is a partitioned table.
+ */
+Relids
+prune_append_rel_partitions(RelOptInfo *rel)
+{
+   Relids      result;
+   List       *clauses = rel->baserestrictinfo;
+   List       *pruning_steps;
+   bool        contradictory;
+   PartitionPruneContext context;
+   Bitmapset  *partindexes;
+   int         i;
+
+   Assert(clauses != NIL);
+   Assert(rel->part_scheme != NULL);
+
+   /* If there are no partitions, return the empty set */
+   if (rel->nparts == 0)
+       return NULL;
+
+   /*
+    * Process clauses.  If the clauses are found to be contradictory, we can
+    * return the empty set.
+    */
+   pruning_steps = gen_partprune_steps(rel, clauses, &contradictory);
+   if (contradictory)
+       return NULL;
+
+   context.strategy = rel->part_scheme->strategy;
+   context.partnatts = rel->part_scheme->partnatts;
+   context.partopfamily = rel->part_scheme->partopfamily;
+   context.partopcintype = rel->part_scheme->partopcintype;
+   context.partcollation = rel->part_scheme->partcollation;
+   context.partsupfunc = rel->part_scheme->partsupfunc;
+   context.nparts = rel->nparts;
+   context.boundinfo = rel->boundinfo;
+
+   /* Actual pruning happens here. */
+   partindexes = get_matching_partitions(&context, pruning_steps);
+
+   /* Add selected partitions' RT indexes to result. */
+   i = -1;
+   result = NULL;
+   while ((i = bms_next_member(partindexes, i)) >= 0)
+       result = bms_add_member(result, rel->part_rels[i]->relid);
+
+   return result;
+}
+
+/*
+ * get_matching_partitions
+ *     Determine partitions that survive partition pruning
+ *
+ * Returns a Bitmapset of indexes of surviving partitions.
+ */
+Bitmapset *
+get_matching_partitions(PartitionPruneContext *context, List *pruning_steps)
+{
+   Bitmapset  *result;
+   int         num_steps = list_length(pruning_steps),
+               i;
+   PruneStepResult **results,
+              *final_result;
+   ListCell   *lc;
+
+   /* If there are no pruning steps then all partitions match. */
+   if (num_steps == 0)
+       return bms_add_range(NULL, 0, context->nparts - 1);
+
+   /*
+    * Allocate space for individual pruning steps to store its result.  Each
+    * slot will hold a PruneStepResult after performing a given pruning step.
+    * Later steps may use the result of one or more earlier steps.  The
+    * result of applying all pruning steps is the value contained in the slot
+    * of the last pruning step.
+    */
+   results = (PruneStepResult **)
+       palloc0(num_steps * sizeof(PruneStepResult *));
+   foreach(lc, pruning_steps)
+   {
+       PartitionPruneStep *step = lfirst(lc);
+
+       switch (nodeTag(step))
+       {
+           case T_PartitionPruneStepOp:
+               results[step->step_id] =
+                   perform_pruning_base_step(context,
+                                             (PartitionPruneStepOp *) step);
+               break;
+
+           case T_PartitionPruneStepCombine:
+               results[step->step_id] =
+                   perform_pruning_combine_step(context,
+                                                (PartitionPruneStepCombine *) step,
+                                                results);
+               break;
+
+           default:
+               elog(ERROR, "invalid pruning step type: %d",
+                    (int) nodeTag(step));
+       }
+   }
+
+   /*
+    * At this point we know the offsets of all the datums whose corresponding
+    * partitions need to be in the result, including special null-accepting
+    * and default partitions.  Collect the actual partition indexes now.
+    */
+   final_result = results[num_steps - 1];
+   Assert(final_result != NULL);
+   i = -1;
+   result = NULL;
+   while ((i = bms_next_member(final_result->bound_offsets, i)) >= 0)
+   {
+       int         partindex = context->boundinfo->indexes[i];
+
+       /*
+        * In range and hash partitioning cases, some slots may contain -1,
+        * indicating that no partition has been defined to accept a given
+        * range of data or for a given remainder, respectively. The default
+        * partition, if any, in case of range partitioning, will be added to
+        * the result, because the specified range still satisfies the query's
+        * conditions.
+        */
+       if (partindex >= 0)
+           result = bms_add_member(result, partindex);
+   }
+
+   /* Add the null and/or default partition if needed and if present. */
+   if (final_result->scan_null)
+   {
+       Assert(context->strategy == PARTITION_STRATEGY_LIST);
+       Assert(partition_bound_accepts_nulls(context->boundinfo));
+       result = bms_add_member(result, context->boundinfo->null_index);
+   }
+   if (final_result->scan_default)
+   {
+       Assert(context->strategy == PARTITION_STRATEGY_LIST ||
+              context->strategy == PARTITION_STRATEGY_RANGE);
+       Assert(partition_bound_has_default(context->boundinfo));
+       result = bms_add_member(result, context->boundinfo->default_index);
+   }
+
+   return result;
+}
+
+/*
+ * gen_partprune_steps_internal
+ *     Processes 'clauses' to generate partition pruning steps.
+ *
+ * From OpExpr clauses that are mutually AND'd, we find combinations of those
+ * that match to the partition key columns and for every such combination,
+ * we emit a PartitionPruneStepOp containing a vector of expressions whose
+ * values are used as a look up key to search partitions by comparing the
+ * values with partition bounds.  Relevant details of the operator and a
+ * vector of (possibly cross-type) comparison functions is also included with
+ * each step.
+ *
+ * For BoolExpr clauses, we recursively generate steps for each argument, and
+ * return a PartitionPruneStepCombine of their results.
+ *
+ * The generated steps are added to the context's steps list.  Each step is
+ * assigned a step identifier, unique even across recursive calls.
+ *
+ * If we find clauses that are mutually contradictory, or a pseudoconstant
+ * 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,
+                            RelOptInfo *rel, List *clauses,
+                            bool *contradictory)
+{
+   PartitionScheme part_scheme = rel->part_scheme;
+   List       *keyclauses[PARTITION_MAX_KEYS];
+   Bitmapset  *nullkeys = NULL,
+              *notnullkeys = NULL;
+   bool        generate_opsteps = false;
+   List       *result = NIL;
+   ListCell   *lc;
+
+   *contradictory = false;
+
+   memset(keyclauses, 0, sizeof(keyclauses));
+   foreach(lc, clauses)
+   {
+       Expr       *clause = (Expr *) lfirst(lc);
+       int         i;
+
+       if (IsA(clause, RestrictInfo))
+       {
+           RestrictInfo *rinfo = (RestrictInfo *) clause;
+
+           clause = rinfo->clause;
+           if (rinfo->pseudoconstant &&
+               IsA(rinfo->clause, Const) &&
+               !DatumGetBool(((Const *) clause)->constvalue))
+           {
+               *contradictory = true;
+               return NIL;
+           }
+       }
+
+       /* Get the BoolExpr's out of the way. */
+       if (IsA(clause, BoolExpr))
+       {
+           /*
+            * Generate steps for arguments.
+            *
+            * While steps generated for the arguments themselves will be
+            * added to context->steps during recursion and will be evaluated
+            * independently, collect their step IDs to be stored in the
+            * combine step we'll be creating.
+            */
+           if (or_clause((Node *) clause))
+           {
+               List       *arg_stepids = NIL;
+               bool        all_args_contradictory = true;
+               ListCell   *lc1;
+
+               /*
+                * Get pruning step for each arg.  If we get contradictory for
+                * all args, it means the OR expression is false as a whole.
+                */
+               foreach(lc1, ((BoolExpr *) clause)->args)
+               {
+                   Expr       *arg = lfirst(lc1);
+                   bool        arg_contradictory;
+                   List       *argsteps;
+
+                   argsteps =
+                       gen_partprune_steps_internal(context, rel,
+                                                    list_make1(arg),
+                                                    &arg_contradictory);
+                   if (!arg_contradictory)
+                       all_args_contradictory = false;
+
+                   if (argsteps != NIL)
+                   {
+                       PartitionPruneStep *step;
+
+                       Assert(list_length(argsteps) == 1);
+                       step = (PartitionPruneStep *) linitial(argsteps);
+                       arg_stepids = lappend_int(arg_stepids, step->step_id);
+                   }
+                   else
+                   {
+                       /*
+                        * No steps either means that arg_contradictory is
+                        * true or the arg didn't contain a clause matching
+                        * this partition key.
+                        *
+                        * In case of the latter, we cannot prune using such
+                        * an arg.  To indicate that to the pruning code, we
+                        * must construct a dummy PartitionPruneStepCombine
+                        * whose source_stepids is set to an empty List.
+                        * However, if we can prove using constraint exclusion
+                        * that the clause refutes the table's partition
+                        * constraint (if it's sub-partitioned), we need not
+                        * bother with that.  That is, we effectively ignore
+                        * this OR arm.
+                        */
+                       List       *partconstr = rel->partition_qual;
+                       PartitionPruneStep *orstep;
+
+                       /* Just ignore this argument. */
+                       if (arg_contradictory)
+                           continue;
+
+                       if (partconstr)
+                       {
+                           partconstr = (List *)
+                               expression_planner((Expr *) partconstr);
+                           if (rel->relid != 1)
+                               ChangeVarNodes((Node *) partconstr, 1,
+                                              rel->relid, 0);
+                           if (predicate_refuted_by(partconstr,
+                                                    list_make1(arg),
+                                                    false))
+                               continue;
+                       }
+
+                       orstep = gen_prune_step_combine(context, NIL,
+                                                       PARTPRUNE_COMBINE_UNION);
+                       arg_stepids = lappend_int(arg_stepids, orstep->step_id);
+                   }
+               }
+
+               *contradictory = all_args_contradictory;
+
+               /* Check if any contradicting clauses were found */
+               if (*contradictory)
+                   return NIL;
+
+               if (arg_stepids != NIL)
+               {
+                   PartitionPruneStep *step;
+
+                   step = gen_prune_step_combine(context, arg_stepids,
+                                                 PARTPRUNE_COMBINE_UNION);
+                   result = lappend(result, step);
+               }
+               continue;
+           }
+           else if (and_clause((Node *) clause))
+           {
+               List       *args = ((BoolExpr *) clause)->args;
+               List       *argsteps,
+                          *arg_stepids = NIL;
+               ListCell   *lc1;
+
+               /*
+                * args may itself contain clauses of arbitrary type, so just
+                * recurse and later combine the component partitions sets
+                * using a combine step.
+                */
+               argsteps = gen_partprune_steps_internal(context, rel, args,
+                                                       contradictory);
+               if (*contradictory)
+                   return NIL;
+
+               foreach(lc1, argsteps)
+               {
+                   PartitionPruneStep *step = lfirst(lc1);
+
+                   arg_stepids = lappend_int(arg_stepids, step->step_id);
+               }
+
+               if (arg_stepids != NIL)
+               {
+                   PartitionPruneStep *step;
+
+                   step = gen_prune_step_combine(context, arg_stepids,
+                                                 PARTPRUNE_COMBINE_INTERSECT);
+                   result = lappend(result, step);
+               }
+               continue;
+           }
+
+           /*
+            * Fall-through for a NOT clause, which if it's a Boolean clause,
+            * will be handled in match_clause_to_partition_key(). We
+            * currently don't perform any pruning for more complex NOT
+            * clauses.
+            */
+       }
+
+       /*
+        * Must be a clause for which we can check if one of its args matches
+        * the partition key.
+        */
+       for (i = 0; i < part_scheme->partnatts; i++)
+       {
+           Expr       *partkey = linitial(rel->partexprs[i]);
+           bool        clause_is_not_null = false;
+           PartClauseInfo *pc = NULL;
+           List       *clause_steps = NIL;
+
+           switch (match_clause_to_partition_key(rel, context,
+                                                 clause, partkey, i,
+                                                 &clause_is_not_null,
+                                                 &pc, &clause_steps))
+           {
+               case PARTCLAUSE_MATCH_CLAUSE:
+                   Assert(pc != NULL);
+
+                   /*
+                    * Since we only allow strict operators, check for any
+                    * contradicting IS NULL.
+                    */
+                   if (bms_is_member(i, nullkeys))
+                   {
+                       *contradictory = true;
+                       return NIL;
+                   }
+                   generate_opsteps = true;
+                   keyclauses[i] = lappend(keyclauses[i], pc);
+                   break;
+
+               case PARTCLAUSE_MATCH_NULLNESS:
+                   if (!clause_is_not_null)
+                   {
+                       /* check for conflicting IS NOT NULL */
+                       if (bms_is_member(i, notnullkeys))
+                       {
+                           *contradictory = true;
+                           return NIL;
+                       }
+                       nullkeys = bms_add_member(nullkeys, i);
+                   }
+                   else
+                   {
+                       /* check for conflicting IS NULL */
+                       if (bms_is_member(i, nullkeys))
+                       {
+                           *contradictory = true;
+                           return NIL;
+                       }
+                       notnullkeys = bms_add_member(notnullkeys, i);
+                   }
+                   break;
+
+               case PARTCLAUSE_MATCH_STEPS:
+                   Assert(clause_steps != NIL);
+                   result = list_concat(result, clause_steps);
+                   break;
+
+               case PARTCLAUSE_MATCH_CONTRADICT:
+                   /* We've nothing more to do if a contradiction was found. */
+                   *contradictory = true;
+                   return NIL;
+
+               case PARTCLAUSE_NOMATCH:
+
+                   /*
+                    * Clause didn't match this key, but it might match the
+                    * next one.
+                    */
+                   continue;
+
+               case PARTCLAUSE_UNSUPPORTED:
+                   /* This clause cannot be used for pruning. */
+                   break;
+
+               default:
+                   Assert(false);
+                   break;
+           }
+
+           /* done; go check the next clause. */
+           break;
+       }
+   }
+
+   /*
+    * If generate_opsteps is set to false it means no OpExprs were directly
+    * present in the input list.
+    */
+   if (!generate_opsteps)
+   {
+       /*
+        * Generate one prune step for the information derived from IS NULL,
+        * if any.  To prune hash partitions, we must have found IS NULL
+        * clauses for all partition keys.
+        */
+       if (!bms_is_empty(nullkeys) &&
+           (part_scheme->strategy != PARTITION_STRATEGY_HASH ||
+            bms_num_members(nullkeys) == part_scheme->partnatts))
+       {
+           PartitionPruneStep *step;
+
+           step = gen_prune_step_op(context, InvalidStrategy,
+                                    false, NIL, NIL, nullkeys);
+           result = lappend(result, step);
+       }
+
+       /*
+        * Note that for IS NOT NULL clauses, simply having step suffices;
+        * there is no need to propagate the exact details of which keys are
+        * required to be NOT NULL.  Hash partitioning expects to see actual
+        * values to perform any pruning.
+        */
+       if (!bms_is_empty(notnullkeys) &&
+           part_scheme->strategy != PARTITION_STRATEGY_HASH)
+       {
+           PartitionPruneStep *step;
+
+           step = gen_prune_step_op(context, InvalidStrategy,
+                                    false, NIL, NIL, NULL);
+           result = lappend(result, step);
+       }
+   }
+   else
+   {
+       PartitionPruneStep *step;
+
+       /* Generate pruning steps from OpExpr clauses in keyclauses. */
+       step = gen_prune_steps_from_opexps(part_scheme, context,
+                                          keyclauses, nullkeys);
+       if (step != NULL)
+           result = lappend(result, step);
+   }
+
+   /*
+    * Finally, results from all entries appearing in result should be
+    * combined using an INTERSECT combine step, if more than one.
+    */
+   if (list_length(result) > 1)
+   {
+       List       *step_ids = NIL;
+
+       foreach(lc, result)
+       {
+           PartitionPruneStep *step = lfirst(lc);
+
+           step_ids = lappend_int(step_ids, step->step_id);
+       }
+
+       if (step_ids != NIL)
+       {
+           PartitionPruneStep *step;
+
+           step = gen_prune_step_combine(context, step_ids,
+                                         PARTPRUNE_COMBINE_INTERSECT);
+           result = lappend(result, step);
+       }
+   }
+
+   return result;
+}
+
+/*
+ * gen_prune_step_op
+ *     Generate a pruning step for a specific operator
+ *
+ * The step is assigned a unique step identifier and added to context's 'steps'
+ * list.
+ */
+static PartitionPruneStep *
+gen_prune_step_op(GeneratePruningStepsContext *context,
+                 StrategyNumber opstrategy, bool op_is_ne,
+                 List *exprs, List *cmpfns,
+                 Bitmapset *nullkeys)
+{
+   PartitionPruneStepOp *opstep = makeNode(PartitionPruneStepOp);
+
+   opstep->step.step_id = context->next_step_id++;
+
+   /*
+    * For clauses that contain an <> operator, set opstrategy to
+    * InvalidStrategy to signal get_matching_list_bounds to do the right
+    * thing.
+    */
+   if (op_is_ne)
+   {
+       Assert(opstrategy == BTEqualStrategyNumber);
+       opstep->opstrategy = InvalidStrategy;
+   }
+   else
+       opstep->opstrategy = opstrategy;
+   Assert(list_length(exprs) == list_length(cmpfns));
+   opstep->exprs = exprs;
+   opstep->cmpfns = cmpfns;
+   opstep->nullkeys = nullkeys;
+
+   context->steps = lappend(context->steps, opstep);
+
+   return (PartitionPruneStep *) opstep;
+}
+
+/*
+ * gen_prune_step_combine
+ *     Generate a pruning step for a combination of several other steps
+ *
+ * The step is assigned a unique step identifier and added to context's
+ * 'steps' list.
+ */
+static PartitionPruneStep *
+gen_prune_step_combine(GeneratePruningStepsContext *context,
+                      List *source_stepids,
+                      PartitionPruneCombineOp combineOp)
+{
+   PartitionPruneStepCombine *cstep = makeNode(PartitionPruneStepCombine);
+
+   cstep->step.step_id = context->next_step_id++;
+   cstep->combineOp = combineOp;
+   cstep->source_stepids = source_stepids;
+
+   context->steps = lappend(context->steps, cstep);
+
+   return (PartitionPruneStep *) cstep;
+}
+
+/*
+ * gen_prune_steps_from_opexps
+ *     Generate pruning steps based on clauses for partition keys
+ *
+ * 'keyclauses' contains one list of clauses per partition key.  We check here
+ * if we have found clauses for a valid subset of the partition key. In some
+ * cases, (depending on the type of partitioning being used) if we didn't
+ * find clauses for a given key, we discard clauses that may have been
+ * found for any subsequent keys; see specific notes below.
+ */
+static PartitionPruneStep *
+gen_prune_steps_from_opexps(PartitionScheme part_scheme,
+                           GeneratePruningStepsContext *context,
+                           List **keyclauses, Bitmapset *nullkeys)
+{
+   ListCell   *lc;
+   List       *opsteps = NIL;
+   List       *btree_clauses[BTMaxStrategyNumber + 1],
+              *hash_clauses[HTMaxStrategyNumber + 1];
+   bool        need_next_less,
+               need_next_eq,
+               need_next_greater;
+   int         i;
+
+   memset(btree_clauses, 0, sizeof(btree_clauses));
+   memset(hash_clauses, 0, sizeof(hash_clauses));
+   for (i = 0; i < part_scheme->partnatts; i++)
+   {
+       List       *clauselist = keyclauses[i];
+       bool        consider_next_key = true;
+
+       /*
+        * To be useful for pruning, we must have clauses for a prefix of
+        * partition keys in the case of range partitioning.  So, ignore
+        * clauses for keys after this one.
+        */
+       if (part_scheme->strategy == PARTITION_STRATEGY_RANGE &&
+           clauselist == NIL)
+           break;
+
+       /*
+        * For hash partitioning, if a column doesn't have the necessary
+        * equality clause, there should be an IS NULL clause, otherwise
+        * pruning is not possible.
+        */
+       if (part_scheme->strategy == PARTITION_STRATEGY_HASH &&
+           clauselist == NIL && !bms_is_member(i, nullkeys))
+           return NULL;
+
+       need_next_eq = need_next_less = need_next_greater = true;
+       foreach(lc, clauselist)
+       {
+           PartClauseInfo *pc = (PartClauseInfo *) lfirst(lc);
+           Oid         lefttype,
+                       righttype;
+
+           /* Look up the operator's btree/hash strategy number. */
+           if (pc->op_strategy == InvalidStrategy)
+               get_op_opfamily_properties(pc->opno,
+                                          part_scheme->partopfamily[i],
+                                          false,
+                                          &pc->op_strategy,
+                                          &lefttype,
+                                          &righttype);
+
+           switch (part_scheme->strategy)
+           {
+               case PARTITION_STRATEGY_LIST:
+               case PARTITION_STRATEGY_RANGE:
+                   {
+                       PartClauseInfo *last = NULL;
+                       bool        inclusive = false;
+
+                       /*
+                        * Add this clause to the list of clauses to be used
+                        * for pruning if this is the first such key for this
+                        * operator strategy or if it is consecutively next to
+                        * the last column for which a clause with this
+                        * operator strategy was matched.
+                        */
+                       if (btree_clauses[pc->op_strategy] != NIL)
+                           last = llast(btree_clauses[pc->op_strategy]);
+
+                       if (last == NULL ||
+                           i == last->keyno || i == last->keyno + 1)
+                           btree_clauses[pc->op_strategy] =
+                               lappend(btree_clauses[pc->op_strategy], pc);
+
+                       /*
+                        * We may not need the next clause if they're of
+                        * certain strategy.
+                        */
+                       switch (pc->op_strategy)
+                       {
+                           case BTLessEqualStrategyNumber:
+                               inclusive = true;
+                               /* fall through */
+                           case BTLessStrategyNumber:
+                               if (!inclusive)
+                                   need_next_eq = need_next_less = false;
+                               break;
+                           case BTEqualStrategyNumber:
+                               /* always accept clauses for the next key. */
+                               break;
+                           case BTGreaterEqualStrategyNumber:
+                               inclusive = true;
+                               /* fall through */
+                           case BTGreaterStrategyNumber:
+                               if (!inclusive)
+                                   need_next_eq = need_next_greater = false;
+                               break;
+                       }
+
+                       /* We may want to change our mind. */
+                       if (consider_next_key)
+                           consider_next_key = (need_next_eq ||
+                                                need_next_less ||
+                                                need_next_greater);
+                       break;
+                   }
+
+               case PARTITION_STRATEGY_HASH:
+                   if (pc->op_strategy != HTEqualStrategyNumber)
+                       elog(ERROR, "invalid clause for hash partitioning");
+                   hash_clauses[pc->op_strategy] =
+                       lappend(hash_clauses[pc->op_strategy], pc);
+                   break;
+
+               default:
+                   elog(ERROR, "invalid partition strategy: %c",
+                        part_scheme->strategy);
+                   break;
+           }
+       }
+
+       /*
+        * If we've decided that clauses for subsequent partition keys
+        * wouldn't be useful for pruning, don't search any further.
+        */
+       if (!consider_next_key)
+           break;
+   }
+
+   /*
+    * Now, we have divided clauses according to their operator strategies.
+    * Check for each strategy if we can generate pruning step(s) by
+    * collecting a list of expressions whose values will constitute a vector
+    * that can be used as a lookup key by a partition bound searching
+    * function.
+    */
+   switch (part_scheme->strategy)
+   {
+       case PARTITION_STRATEGY_LIST:
+       case PARTITION_STRATEGY_RANGE:
+           {
+               List       *eq_clauses = btree_clauses[BTEqualStrategyNumber];
+               List       *le_clauses = btree_clauses[BTLessEqualStrategyNumber];
+               List       *ge_clauses = btree_clauses[BTGreaterEqualStrategyNumber];
+               int         strat;
+
+               /*
+                * For each clause under consideration for a given strategy,
+                * we collect expressions from clauses for earlier keys, whose
+                * operator strategy is inclusive, into a list called
+                * 'prefix'. By appending the clause's own expression to the
+                * 'prefix', we'll generate one step using the so generated
+                * vector and assign the current strategy to it.  Actually,
+                * 'prefix' might contain multiple clauses for the same key,
+                * in which case, we must generate steps for various
+                * combinations of expressions of different keys, which
+                * get_steps_using_prefix takes care of for us.
+                */
+               for (strat = 1; strat <= BTMaxStrategyNumber; strat++)
+               {
+                   foreach(lc, btree_clauses[strat])
+                   {
+                       PartClauseInfo *pc = lfirst(lc);
+                       ListCell   *lc1;
+                       List       *prefix = NIL;
+                       List       *pc_steps;
+
+                       /*
+                        * Expressions from = clauses can always be in the
+                        * prefix, provided they're from an earlier key.
+                        */
+                       foreach(lc1, eq_clauses)
+                       {
+                           PartClauseInfo *eqpc = lfirst(lc1);
+
+                           if (eqpc->keyno == pc->keyno)
+                               break;
+                           if (eqpc->keyno < pc->keyno)
+                               prefix = lappend(prefix, eqpc);
+                       }
+
+                       /*
+                        * If we're generating steps for 
+                        * add other <= clauses to the prefix, provided
+                        * they're from an earlier key.
+                        */
+                       if (strat == BTLessStrategyNumber ||
+                           strat == BTLessEqualStrategyNumber)
+                       {
+                           foreach(lc1, le_clauses)
+                           {
+                               PartClauseInfo *lepc = lfirst(lc1);
+
+                               if (lepc->keyno == pc->keyno)
+                                   break;
+                               if (lepc->keyno < pc->keyno)
+                                   prefix = lappend(prefix, lepc);
+                           }
+                       }
+
+                       /*
+                        * If we're generating steps for >/>= strategy, we can
+                        * add other >= clauses to the prefix, provided
+                        * they're from an earlier key.
+                        */
+                       if (strat == BTGreaterStrategyNumber ||
+                           strat == BTGreaterEqualStrategyNumber)
+                       {
+                           foreach(lc1, ge_clauses)
+                           {
+                               PartClauseInfo *gepc = lfirst(lc1);
+
+                               if (gepc->keyno == pc->keyno)
+                                   break;
+                               if (gepc->keyno < pc->keyno)
+                                   prefix = lappend(prefix, gepc);
+                           }
+                       }
+
+                       /*
+                        * As mentioned above, if 'prefix' contains multiple
+                        * expressions for the same key, the following will
+                        * generate multiple steps, one for each combination
+                        * of the expressions for different keys.
+                        *
+                        * Note that we pass NULL for step_nullkeys, because
+                        * we don't search list/range partition bounds where
+                        * some keys are NULL.
+                        */
+                       Assert(pc->op_strategy == strat);
+                       pc_steps = get_steps_using_prefix(context, strat,
+                                                         pc->op_is_ne,
+                                                         pc->expr,
+                                                         pc->cmpfn,
+                                                         pc->keyno,
+                                                         NULL,
+                                                         prefix);
+                       opsteps = list_concat(opsteps, list_copy(pc_steps));
+                   }
+               }
+               break;
+           }
+
+       case PARTITION_STRATEGY_HASH:
+           {
+               List       *eq_clauses = hash_clauses[HTEqualStrategyNumber];
+
+               /* For hash partitioning, we have just the = strategy. */
+               if (eq_clauses != NIL)
+               {
+                   PartClauseInfo *pc;
+                   List       *pc_steps;
+                   List       *prefix = NIL;
+                   int         last_keyno;
+                   ListCell   *lc1;
+
+                   /*
+                    * Locate the clause for the greatest column.  This may
+                    * not belong to the last partition key, but it is the
+                    * clause belonging to the last partition key we found a
+                    * clause for above.
+                    */
+                   pc = llast(eq_clauses);
+
+                   /*
+                    * There might be multiple clauses which matched to that
+                    * partition key; find the first such clause.  While at
+                    * it, add all the clauses before that one to 'prefix'.
+                    */
+                   last_keyno = pc->keyno;
+                   foreach(lc, eq_clauses)
+                   {
+                       pc = lfirst(lc);
+                       if (pc->keyno == last_keyno)
+                           break;
+                       prefix = lappend(prefix, pc);
+                   }
+
+                   /*
+                    * For each clause for the "last" column, after appending
+                    * the clause's own expression to the 'prefix', we'll
+                    * generate one step using the so generated vector and and
+                    * assign = as its strategy.  Actually, 'prefix' might
+                    * contain multiple clauses for the same key, in which
+                    * case, we must generate steps for various combinations
+                    * of expressions of different keys, which
+                    * get_steps_using_prefix will take care of for us.
+                    */
+                   for_each_cell(lc1, lc)
+                   {
+                       pc = lfirst(lc1);
+
+                       /*
+                        * Note that we pass nullkeys for step_nullkeys,
+                        * because we need to tell hash partition bound search
+                        * function which of the keys we found IS NULL clauses
+                        * for.
+                        */
+                       Assert(pc->op_strategy == HTEqualStrategyNumber);
+                       pc_steps =
+                           get_steps_using_prefix(context,
+                                                  HTEqualStrategyNumber,
+                                                  false,
+                                                  pc->expr,
+                                                  pc->cmpfn,
+                                                  pc->keyno,
+                                                  nullkeys,
+                                                  prefix);
+                       opsteps = list_concat(opsteps, list_copy(pc_steps));
+                   }
+               }
+               break;
+           }
+
+       default:
+           elog(ERROR, "invalid partition strategy: %c",
+                part_scheme->strategy);
+           break;
+   }
+
+   /* Lastly, add a combine step to mutually AND these op steps, if needed */
+   if (list_length(opsteps) > 1)
+   {
+       List       *opstep_ids = NIL;
+
+       foreach(lc, opsteps)
+       {
+           PartitionPruneStep *step = lfirst(lc);
+
+           opstep_ids = lappend_int(opstep_ids, step->step_id);
+       }
+
+       if (opstep_ids != NIL)
+           return gen_prune_step_combine(context, opstep_ids,
+                                         PARTPRUNE_COMBINE_INTERSECT);
+       return NULL;
+   }
+   else if (opsteps != NIL)
+       return linitial(opsteps);
+
+   return NULL;
+}
+
+/*
+ * If the partition key has a collation, then the clause must have the same
+ * input collation.  If the partition key is non-collatable, we assume the
+ * collation doesn't matter, because while collation wasn't considered when
+ * performing partitioning, the clause still may have a collation assigned
+ * due to the other input being of a collatable type.
+ *
+ * See also IndexCollMatchesExprColl.
+ */
+#define PartCollMatchesExprColl(partcoll, exprcoll) \
+   ((partcoll) == InvalidOid || (partcoll) == (exprcoll))
+
+/*
+ * match_clause_to_partition_key
+ *     Attempt to match the given 'clause' with the specified partition key.
+ *
+ * Return value is:
+ * * PARTCLAUSE_NOMATCH if the clause doesn't match this partition key (but
+ *   caller should keep trying, because it might match a subsequent key).
+ *   Output arguments: none set.
+ *
+ * * PARTCLAUSE_MATCH_CLAUSE if there is a match.
+ *   Output arguments: *pc is set to a PartClauseInfo constructed for the
+ *   matched clause.
+ *
+ * * PARTCLAUSE_MATCH_NULLNESS if there is a match, and the matched clause was
+ *   either a "a IS NULL" or "a IS NOT NULL" clause.
+ *   Output arguments: *clause_is_not_null is set to false in the former case
+ *   true otherwise.
+ *
+ * * PARTCLAUSE_MATCH_STEPS if there is a match.
+ *   Output arguments: *clause_steps is set to a list of PartitionPruneStep
+ *   generated for the clause.
+ *
+ * * PARTCLAUSE_MATCH_CONTRADICT if the clause is self-contradictory.  This can
+ *   only happen if it's a BoolExpr whose arguments are self-contradictory.
+ *   Output arguments: none set.
+ *
+ * * PARTCLAUSE_UNSUPPORTED if the clause cannot be used for pruning at all
+ *   due to one of its properties, such as argument volatility, even if it may
+ *   have been matched with a key.
+ *   Output arguments: none set.
+ */
+static PartClauseMatchStatus
+match_clause_to_partition_key(RelOptInfo *rel,
+                             GeneratePruningStepsContext *context,
+                             Expr *clause, Expr *partkey, int partkeyidx,
+                             bool *clause_is_not_null, PartClauseInfo **pc,
+                             List **clause_steps)
+{
+   PartitionScheme part_scheme = rel->part_scheme;
+   Expr       *expr;
+   Oid         partopfamily = part_scheme->partopfamily[partkeyidx],
+               partcoll = part_scheme->partcollation[partkeyidx];
+
+   /*
+    * Recognize specially shaped clauses that match with the Boolean
+    * partition key.
+    */
+   if (match_boolean_partition_clause(partopfamily, clause, partkey, &expr))
+   {
+       PartClauseInfo *partclause;
+
+       partclause = (PartClauseInfo *) palloc(sizeof(PartClauseInfo));
+       partclause->keyno = partkeyidx;
+       /* Do pruning with the Boolean equality operator. */
+       partclause->opno = BooleanEqualOperator;
+       partclause->op_is_ne = false;
+       partclause->expr = expr;
+       /* We know that expr is of Boolean type. */
+       partclause->cmpfn = rel->part_scheme->partsupfunc[partkeyidx].fn_oid;
+       partclause->op_strategy = InvalidStrategy;
+
+       *pc = partclause;
+
+       return PARTCLAUSE_MATCH_CLAUSE;
+   }
+   else if (IsA(clause, OpExpr) &&
+            list_length(((OpExpr *) clause)->args) == 2)
+   {
+       OpExpr     *opclause = (OpExpr *) clause;
+       Expr       *leftop,
+                  *rightop;
+       Oid         commutator = InvalidOid,
+                   negator = InvalidOid;
+       Oid         cmpfn;
+       Oid         exprtype;
+       bool        is_opne_listp = false;
+       PartClauseInfo *partclause;
+
+       leftop = (Expr *) get_leftop(clause);
+       if (IsA(leftop, RelabelType))
+           leftop = ((RelabelType *) leftop)->arg;
+       rightop = (Expr *) get_rightop(clause);
+       if (IsA(rightop, RelabelType))
+           rightop = ((RelabelType *) rightop)->arg;
+
+       /* check if the clause matches this partition key */
+       if (equal(leftop, partkey))
+           expr = rightop;
+       else if (equal(rightop, partkey))
+       {
+           expr = leftop;
+           commutator = get_commutator(opclause->opno);
+
+           /* nothing we can do unless we can swap the operands */
+           if (!OidIsValid(commutator))
+               return PARTCLAUSE_UNSUPPORTED;
+       }
+       else
+           /* clause does not match this partition key, but perhaps next. */
+           return PARTCLAUSE_NOMATCH;
+
+       /*
+        * Partition key also consists of a collation that's specified for it,
+        * so try to match it too.  There may be multiple keys with the same
+        * expression but different collations.
+        */
+       if (!PartCollMatchesExprColl(partcoll, opclause->inputcollid))
+           return PARTCLAUSE_NOMATCH;
+
+       /*
+        * Matched with this key.  Now check various properties of the clause
+        * to see if it's sane to use it for pruning.  If any of the
+        * properties makes it unsuitable for pruning, then the clause is
+        * useless no matter which key it's matched to.
+        */
+
+       /*
+        * Only allow strict operators.  This will guarantee nulls are
+        * filtered.
+        */
+       if (!op_strict(opclause->opno))
+           return PARTCLAUSE_UNSUPPORTED;
+
+       /* We can't use any volatile expressions to prune partitions. */
+       if (contain_volatile_functions((Node *) expr))
+           return PARTCLAUSE_UNSUPPORTED;
+
+       /*
+        * Normally we only bother with operators that are listed as being
+        * part of the partitioning operator family.  But we make an exception
+        * in one case -- operators named '<>' are not listed in any operator
+        * family whatsoever, in which case, we try to perform partition
+        * pruning with it only if list partitioning is in use.
+        */
+       if (!op_in_opfamily(opclause->opno, partopfamily))
+       {
+           if (part_scheme->strategy != PARTITION_STRATEGY_LIST)
+               return PARTCLAUSE_UNSUPPORTED;
+
+           /*
+            * To confirm if the operator is really '<>', check if its negator
+            * is a btree equality operator.
+            */
+           negator = get_negator(opclause->opno);
+           if (OidIsValid(negator) && op_in_opfamily(negator, partopfamily))
+           {
+               Oid         lefttype;
+               Oid         righttype;
+               int         strategy;
+
+               get_op_opfamily_properties(negator, partopfamily, false,
+                                          &strategy, &lefttype, &righttype);
+
+               if (strategy == BTEqualStrategyNumber)
+                   is_opne_listp = true;
+           }
+
+           /* Operator isn't really what we were hoping it'd be. */
+           if (!is_opne_listp)
+               return PARTCLAUSE_UNSUPPORTED;
+       }
+
+       /* Check if we're going to need a cross-type comparison function. */
+       exprtype = exprType((Node *) expr);
+       if (exprtype != part_scheme->partopcintype[partkeyidx])
+       {
+           switch (part_scheme->strategy)
+           {
+               case PARTITION_STRATEGY_LIST:
+               case PARTITION_STRATEGY_RANGE:
+                   cmpfn =
+                       get_opfamily_proc(part_scheme->partopfamily[partkeyidx],
+                                         part_scheme->partopcintype[partkeyidx],
+                                         exprtype, BTORDER_PROC);
+                   break;
+
+               case PARTITION_STRATEGY_HASH:
+                   cmpfn =
+                       get_opfamily_proc(part_scheme->partopfamily[partkeyidx],
+                                         exprtype, exprtype, HASHEXTENDED_PROC);
+                   break;
+
+               default:
+                   elog(ERROR, "invalid partition strategy: %c",
+                        part_scheme->strategy);
+                   break;
+           }
+
+           /* If we couldn't find one, we cannot use this expression. */
+           if (!OidIsValid(cmpfn))
+               return PARTCLAUSE_UNSUPPORTED;
+       }
+       else
+           cmpfn = part_scheme->partsupfunc[partkeyidx].fn_oid;
+
+       partclause = (PartClauseInfo *) palloc(sizeof(PartClauseInfo));
+       partclause->keyno = partkeyidx;
+
+       /* For <> operator clauses, pass on the negator. */
+       partclause->op_is_ne = false;
+       partclause->op_strategy = InvalidStrategy;
+
+       if (is_opne_listp)
+       {
+           Assert(OidIsValid(negator));
+           partclause->opno = negator;
+           partclause->op_is_ne = true;
+
+           /*
+            * We already know the strategy in this case, so may as well set
+            * it rather than having to look it up later.
+            */
+           partclause->op_strategy = BTEqualStrategyNumber;
+       }
+       /* And if commuted before matching, pass on the commutator */
+       else if (OidIsValid(commutator))
+           partclause->opno = commutator;
+       else
+           partclause->opno = opclause->opno;
+
+       partclause->expr = expr;
+       partclause->cmpfn = cmpfn;
+
+       *pc = partclause;
+
+       return PARTCLAUSE_MATCH_CLAUSE;
+   }
+   else if (IsA(clause, ScalarArrayOpExpr))
+   {
+       ScalarArrayOpExpr *saop = (ScalarArrayOpExpr *) clause;
+       Oid         saop_op = saop->opno;
+       Oid         saop_coll = saop->inputcollid;
+       Expr       *leftop = (Expr *) linitial(saop->args),
+                  *rightop = (Expr *) lsecond(saop->args);
+       List       *elem_exprs,
+                  *elem_clauses;
+       ListCell   *lc1;
+
+       if (IsA(leftop, RelabelType))
+           leftop = ((RelabelType *) leftop)->arg;
+
+       /* Check it 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.
+        */
+
+       /*
+        * Only allow strict operators.  This will guarantee nulls are
+        * filtered.
+        */
+       if (!op_strict(saop->opno))
+           return PARTCLAUSE_UNSUPPORTED;
+
+       /* Useless if the array has any volatile functions. */
+       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
+        * is a btree equality operator belonging to the partitioning operator
+        * family.
+        */
+       if (!op_in_opfamily(saop_op, partopfamily))
+       {
+           Oid         negator;
+
+           if (part_scheme->strategy != PARTITION_STRATEGY_LIST)
+               return PARTCLAUSE_UNSUPPORTED;
+
+           negator = get_negator(saop_op);
+           if (OidIsValid(negator) && op_in_opfamily(negator, partopfamily))
+           {
+               int         strategy;
+               Oid         lefttype,
+                           righttype;
+
+               get_op_opfamily_properties(negator, partopfamily,
+                                          false, &strategy,
+                                          &lefttype, &righttype);
+               if (strategy != BTEqualStrategyNumber)
+                   return PARTCLAUSE_UNSUPPORTED;
+           }
+       }
+
+       /*
+        * First generate a list of Const nodes, one for each array element
+        * (excepting nulls).
+        */
+       elem_exprs = NIL;
+       if (IsA(rightop, Const))
+       {
+           Const      *arr = castNode(Const, rightop);
+           ArrayType  *arrval = DatumGetArrayTypeP(arr->constvalue);
+           int16       elemlen;
+           bool        elembyval;
+           char        elemalign;
+           Datum      *elem_values;
+           bool       *elem_nulls;
+           int         num_elems,
+                       i;
+
+           get_typlenbyvalalign(ARR_ELEMTYPE(arrval),
+                                &elemlen, &elembyval, &elemalign);
+           deconstruct_array(arrval,
+                             ARR_ELEMTYPE(arrval),
+                             elemlen, elembyval, elemalign,
+                             &elem_values, &elem_nulls,
+                             &num_elems);
+           for (i = 0; i < num_elems; i++)
+           {
+               Const      *elem_expr;
+
+               /* Only consider non-null values. */
+               if (elem_nulls[i])
+                   continue;
+
+               elem_expr = makeConst(ARR_ELEMTYPE(arrval), -1,
+                                     arr->constcollid, elemlen,
+                                     elem_values[i], false, elembyval);
+               elem_exprs = lappend(elem_exprs, elem_expr);
+           }
+       }
+       else
+       {
+           ArrayExpr  *arrexpr = castNode(ArrayExpr, rightop);
+
+           /*
+            * For a nested ArrayExpr, we don't know how to get the actual
+            * scalar values out into a flat list, so we give up doing
+            * anything with this ScalarArrayOpExpr.
+            */
+           if (arrexpr->multidims)
+               return PARTCLAUSE_UNSUPPORTED;
+
+           elem_exprs = arrexpr->elements;
+       }
+
+       /*
+        * Now generate a list of clauses, one for each array element, of the
+        * form saop_leftop saop_op elem_expr
+        */
+       elem_clauses = NIL;
+       foreach(lc1, elem_exprs)
+       {
+           Expr       *rightop = (Expr *) lfirst(lc1),
+                      *elem_clause;
+
+           elem_clause = make_opclause(saop_op, BOOLOID, false,
+                                       leftop, rightop,
+                                       InvalidOid, saop_coll);
+           elem_clauses = lappend(elem_clauses, elem_clause);
+       }
+
+       /*
+        * Build a combine step as if for an OR clause or add the clauses to
+        * the end of the list that's being processed currently.
+        */
+       if (saop->useOr && list_length(elem_clauses) > 1)
+       {
+           Expr       *orexpr;
+           bool        contradictory;
+
+           orexpr = makeBoolExpr(OR_EXPR, elem_clauses, -1);
+           *clause_steps =
+               gen_partprune_steps_internal(context, rel, list_make1(orexpr),
+                                            &contradictory);
+           if (contradictory)
+               return PARTCLAUSE_MATCH_CONTRADICT;
+
+           Assert(list_length(*clause_steps) == 1);
+           return PARTCLAUSE_MATCH_STEPS;
+       }
+       else
+       {
+           bool        contradictory;
+
+           *clause_steps =
+               gen_partprune_steps_internal(context, rel, elem_clauses,
+                                            &contradictory);
+           if (contradictory)
+               return PARTCLAUSE_MATCH_CONTRADICT;
+           Assert(list_length(*clause_steps) >= 1);
+           return PARTCLAUSE_MATCH_STEPS;
+       }
+   }
+   else if (IsA(clause, NullTest))
+   {
+       NullTest   *nulltest = (NullTest *) clause;
+       Expr       *arg = nulltest->arg;
+
+       if (IsA(arg, RelabelType))
+           arg = ((RelabelType *) arg)->arg;
+
+       /* Does arg match with this partition key column? */
+       if (!equal(arg, partkey))
+           return PARTCLAUSE_NOMATCH;
+
+       *clause_is_not_null = nulltest->nulltesttype == IS_NOT_NULL;
+
+       return PARTCLAUSE_MATCH_NULLNESS;
+   }
+
+   return PARTCLAUSE_UNSUPPORTED;
+}
+
+/*
+ * get_steps_using_prefix
+ *     Generate list of PartitionPruneStepOp steps each consisting of given
+ *     opstrategy
+ *
+ * To generate steps, step_lastexpr and step_lastcmpfn are appended to
+ * expressions and cmpfns, respectively, extracted from the clauses in
+ * 'prefix'.  Actually, since 'prefix' may contain multiple clauses for the
+ * same partition key column, we must generate steps for various combinations
+ * of the clauses of different keys.
+ */
+static List *
+get_steps_using_prefix(GeneratePruningStepsContext *context,
+                      StrategyNumber step_opstrategy,
+                      bool step_op_is_ne,
+                      Expr *step_lastexpr,
+                      Oid step_lastcmpfn,
+                      int step_lastkeyno,
+                      Bitmapset *step_nullkeys,
+                      List *prefix)
+{
+   /* Quick exit if there are no values to prefix with. */
+   if (list_length(prefix) == 0)
+   {
+       PartitionPruneStep *step;
+
+       step = gen_prune_step_op(context,
+                                step_opstrategy,
+                                step_op_is_ne,
+                                list_make1(step_lastexpr),
+                                list_make1_oid(step_lastcmpfn),
+                                step_nullkeys);
+       return list_make1(step);
+   }
+
+   /* Recurse to generate steps for various combinations. */
+   return get_steps_using_prefix_recurse(context,
+                                         step_opstrategy,
+                                         step_op_is_ne,
+                                         step_lastexpr,
+                                         step_lastcmpfn,
+                                         step_lastkeyno,
+                                         step_nullkeys,
+                                         list_head(prefix),
+                                         NIL, NIL);
+}
+
+/*
+ * get_steps_using_prefix_recurse
+ *     Recursively generate combinations of clauses for different partition
+ *     keys and start generating steps upon reaching clauses for the greatest
+ *     column that is less than the one for which we're currently generating
+ *     steps (that is, step_lastkeyno)
+ *
+ * 'start' is where we should start iterating for the current invocation.
+ * 'step_exprs' and 'step_cmpfns' each contains the expressions and cmpfns
+ * we've generated so far from the clauses for the previous part keys.
+ */
+static List *
+get_steps_using_prefix_recurse(GeneratePruningStepsContext *context,
+                              StrategyNumber step_opstrategy,
+                              bool step_op_is_ne,
+                              Expr *step_lastexpr,
+                              Oid step_lastcmpfn,
+                              int step_lastkeyno,
+                              Bitmapset *step_nullkeys,
+                              ListCell *start,
+                              List *step_exprs,
+                              List *step_cmpfns)
+{
+   List       *result = NIL;
+   ListCell   *lc;
+   int         cur_keyno;
+
+   /* Actually, recursion would be limited by PARTITION_MAX_KEYS. */
+   check_stack_depth();
+
+   /* Check if we need to recurse. */
+   Assert(start != NULL);
+   cur_keyno = ((PartClauseInfo *) lfirst(start))->keyno;
+   if (cur_keyno < step_lastkeyno - 1)
+   {
+       PartClauseInfo *pc;
+       ListCell   *next_start;
+
+       /*
+        * For each clause with cur_keyno, adds its expr and cmpfn to
+        * step_exprs and step_cmpfns, respectively, and recurse after setting
+        * next_start to the ListCell of the first clause for the next
+        * partition key.
+        */
+       for_each_cell(lc, start)
+       {
+           pc = lfirst(lc);
+
+           if (pc->keyno > cur_keyno)
+               break;
+       }
+       next_start = lc;
+
+       for_each_cell(lc, start)
+       {
+           List       *moresteps;
+
+           pc = lfirst(lc);
+           if (pc->keyno == cur_keyno)
+           {
+               /* clean up before starting a new recursion cycle. */
+               if (cur_keyno == 0)
+               {
+                   list_free(step_exprs);
+                   list_free(step_cmpfns);
+                   step_exprs = list_make1(pc->expr);
+                   step_cmpfns = list_make1_oid(pc->cmpfn);
+               }
+               else
+               {
+                   step_exprs = lappend(step_exprs, pc->expr);
+                   step_cmpfns = lappend_oid(step_cmpfns, pc->cmpfn);
+               }
+           }
+           else
+           {
+               Assert(pc->keyno > cur_keyno);
+               break;
+           }
+
+           moresteps = get_steps_using_prefix_recurse(context,
+                                                      step_opstrategy,
+                                                      step_op_is_ne,
+                                                      step_lastexpr,
+                                                      step_lastcmpfn,
+                                                      step_lastkeyno,
+                                                      step_nullkeys,
+                                                      next_start,
+                                                      step_exprs,
+                                                      step_cmpfns);
+           result = list_concat(result, moresteps);
+       }
+   }
+   else
+   {
+       /*
+        * End the current recursion cycle and start generating steps, one for
+        * each clause with cur_keyno, which is all clauses from here onward
+        * till the end of the list.
+        */
+       Assert(list_length(step_exprs) == cur_keyno);
+       for_each_cell(lc, start)
+       {
+           PartClauseInfo *pc = lfirst(lc);
+           PartitionPruneStep *step;
+           List       *step_exprs1,
+                      *step_cmpfns1;
+
+           Assert(pc->keyno == cur_keyno);
+
+           /* Leave the original step_exprs unmodified. */
+           step_exprs1 = list_copy(step_exprs);
+           step_exprs1 = lappend(step_exprs1, pc->expr);
+           step_exprs1 = lappend(step_exprs1, step_lastexpr);
+
+           /* Leave the original step_cmpfns unmodified. */
+           step_cmpfns1 = list_copy(step_cmpfns);
+           step_cmpfns1 = lappend_oid(step_cmpfns1, pc->cmpfn);
+           step_cmpfns1 = lappend_oid(step_cmpfns1, step_lastcmpfn);
+
+           step = gen_prune_step_op(context,
+                                    step_opstrategy, step_op_is_ne,
+                                    step_exprs1, step_cmpfns1,
+                                    step_nullkeys);
+           result = lappend(result, step);
+       }
+   }
+
+   return result;
+}
+
+/*
+ * get_matching_hash_bounds
+ *     Determine offset of the hash bound matching the specified values,
+ *     considering that all the non-null values come from clauses containing
+ *     a compatible hash equality operator and any keys that are null come
+ *     from an IS NULL clause.
+ *
+ * Generally this function will return a single matching bound offset,
+ * although if a partition has not been setup for a given modulus then we may
+ * return no matches.  If the number of clauses found don't cover the entire
+ * partition key, then we'll need to return all offsets.
+ *
+ * 'opstrategy' if non-zero must be HTEqualStrategyNumber.
+ *
+ * 'values' contains Datums indexed by the partition key to use for pruning.
+ *
+ * 'nvalues', the number of Datums in the 'values' array.
+ *
+ * 'partsupfunc' contains partition hashing functions that can produce correct
+ * hash for the type of the values contained in 'values'.
+ *
+ * 'nullkeys' is the set of partition keys that are null.
+ */
+static PruneStepResult *
+get_matching_hash_bounds(PartitionPruneContext *context,
+                        StrategyNumber opstrategy, Datum *values, int nvalues,
+                        FmgrInfo *partsupfunc, Bitmapset *nullkeys)
+{
+   PruneStepResult *result = (PruneStepResult *) palloc0(sizeof(PruneStepResult));
+   PartitionBoundInfo boundinfo = context->boundinfo;
+   int        *partindices = boundinfo->indexes;
+   int         partnatts = context->partnatts;
+   bool        isnull[PARTITION_MAX_KEYS];
+   int         i;
+   uint64      rowHash;
+   int         greatest_modulus;
+
+   Assert(context->strategy == PARTITION_STRATEGY_HASH);
+
+   /*
+    * For hash partitioning we can only perform pruning based on equality
+    * clauses to the partition key or IS NULL clauses.  We also can only
+    * prune if we got values for all keys.
+    */
+   if (nvalues + bms_num_members(nullkeys) == partnatts)
+   {
+       /*
+        * If there are any values, they must have come from clauses
+        * containing an equality operator compatible with hash partitioning.
+        */
+       Assert(opstrategy == HTEqualStrategyNumber || nvalues == 0);
+
+       for (i = 0; i < partnatts; i++)
+           isnull[i] = bms_is_member(i, nullkeys);
+
+       greatest_modulus = get_hash_partition_greatest_modulus(boundinfo);
+       rowHash = compute_hash_value(partnatts, partsupfunc, values, isnull);
+
+       if (partindices[rowHash % greatest_modulus] >= 0)
+           result->bound_offsets =
+               bms_make_singleton(rowHash % greatest_modulus);
+   }
+   else
+       result->bound_offsets = bms_add_range(NULL, 0,
+                                             boundinfo->ndatums - 1);
+
+   /*
+    * There is neither a special hash null partition or the default hash
+    * partition.
+    */
+   result->scan_null = result->scan_default = false;
+
+   return result;
+}
+
+/*
+ * get_matching_list_bounds
+ *     Determine the offsets of list bounds matching the specified value,
+ *     according to the semantics of the given operator strategy
+ * 'opstrategy' if non-zero must be a btree strategy number.
+ *
+ * 'value' contains the value to use for pruning.
+ *
+ * 'nvalues', if non-zero, should be exactly 1, because of list partitioning.
+ *
+ * 'partsupfunc' contains the list partitioning comparison function to be used
+ * to perform partition_list_bsearch
+ *
+ * 'nullkeys' is the set of partition keys that are null.
+ */
+static PruneStepResult *
+get_matching_list_bounds(PartitionPruneContext *context,
+                        StrategyNumber opstrategy, Datum value, int nvalues,
+                        FmgrInfo *partsupfunc, Bitmapset *nullkeys)
+{
+   PruneStepResult *result = (PruneStepResult *) palloc0(sizeof(PruneStepResult));
+   PartitionBoundInfo boundinfo = context->boundinfo;
+   int         off,
+               minoff,
+               maxoff;
+   bool        is_equal;
+   bool        inclusive = false;
+   Oid        *partcollation = context->partcollation;
+
+   Assert(context->strategy == PARTITION_STRATEGY_LIST);
+   Assert(context->partnatts == 1);
+
+   result->scan_null = result->scan_default = false;
+
+   if (!bms_is_empty(nullkeys))
+   {
+       /*
+        * Nulls may exist in only one partition - the partition whose
+        * accepted set of values includes null or the default partition if
+        * the former doesn't exist.
+        */
+       if (partition_bound_accepts_nulls(boundinfo))
+           result->scan_null = true;
+       else
+           result->scan_default = partition_bound_has_default(boundinfo);
+       return result;
+   }
+
+   /*
+    * If there are no datums to compare keys with, but there are partitions,
+    * just return the default partition if one exists.
+    */
+   if (boundinfo->ndatums == 0)
+   {
+       result->scan_default = partition_bound_has_default(boundinfo);
+       return result;
+   }
+
+   minoff = 0;
+   maxoff = boundinfo->ndatums - 1;
+
+   /*
+    * If there are no values to compare with the datums in boundinfo, it
+    * means the caller asked for partitions for all non-null datums.  Add
+    * indexes of *all* partitions, including the default if any.
+    */
+   if (nvalues == 0)
+   {
+       result->bound_offsets = bms_add_range(NULL, 0,
+                                             boundinfo->ndatums - 1);
+       result->scan_default = partition_bound_has_default(boundinfo);
+       return result;
+   }
+
+   /* Special case handling of values coming from a <> operator clause. */
+   if (opstrategy == InvalidStrategy)
+   {
+       /*
+        * First match to all bounds.  We'll remove any matching datums below.
+        */
+       result->bound_offsets = bms_add_range(NULL, 0,
+                                             boundinfo->ndatums - 1);
+
+       off = partition_list_bsearch(partsupfunc, partcollation, boundinfo,
+                                    value, &is_equal);
+       if (off >= 0 && is_equal)
+       {
+
+           /* We have a match. Remove from the result. */
+           Assert(boundinfo->indexes[off] >= 0);
+           result->bound_offsets = bms_del_member(result->bound_offsets,
+                                                  off);
+       }
+
+       /* Always include the default partition if any. */
+       result->scan_default = partition_bound_has_default(boundinfo);
+
+       return result;
+   }
+
+   /*
+    * With range queries, always include the default list partition, because
+    * list partitions divide the key space in a discontinuous manner, not all
+    * values in the given range will have a partition assigned.  This may not
+    * technically be true for some data types (e.g. integer types), however,
+    * we currently lack any sort of infrastructure to provide us with proofs
+    * that would allow us to do anything smarter here.
+    */
+   if (opstrategy != BTEqualStrategyNumber)
+       result->scan_default = partition_bound_has_default(boundinfo);
+
+   switch (opstrategy)
+   {
+       case BTEqualStrategyNumber:
+           off = partition_list_bsearch(partsupfunc,
+                                        partcollation,
+                                        boundinfo, value,
+                                        &is_equal);
+           if (off >= 0 && is_equal)
+           {
+               Assert(boundinfo->indexes[off] >= 0);
+               result->bound_offsets = bms_make_singleton(off);
+           }
+           else
+               result->scan_default = partition_bound_has_default(boundinfo);
+           return result;
+
+       case BTGreaterEqualStrategyNumber:
+           inclusive = true;
+           /* fall through */
+       case BTGreaterStrategyNumber:
+           off = partition_list_bsearch(partsupfunc,
+                                        partcollation,
+                                        boundinfo, value,
+                                        &is_equal);
+           if (off >= 0)
+           {
+               /* We don't want the matched datum to be in the result. */
+               if (!is_equal || !inclusive)
+                   off++;
+           }
+           else
+           {
+               /*
+                * This case means all partition bounds are greater, which in
+                * turn means that all partitions satisfy this key.
+                */
+               off = 0;
+           }
+
+           /*
+            * off is greater than the numbers of datums we have partitions
+            * for.  The only possible partition that could contain a match is
+            * the default partition, but we must've set context->scan_default
+            * above anyway if one exists.
+            */
+           if (off > boundinfo->ndatums - 1)
+               return result;
+
+           minoff = off;
+           break;
+
+       case BTLessEqualStrategyNumber:
+           inclusive = true;
+           /* fall through */
+       case BTLessStrategyNumber:
+           off = partition_list_bsearch(partsupfunc,
+                                        partcollation,
+                                        boundinfo, value,
+                                        &is_equal);
+           if (off >= 0 && is_equal && !inclusive)
+               off--;
+
+           /*
+            * off is smaller than the datums of all non-default partitions.
+            * The only possible partition that could contain a match is the
+            * default partition, but we must've set context->scan_default
+            * above anyway if one exists.
+            */
+           if (off < 0)
+               return result;
+
+           maxoff = off;
+           break;
+
+       default:
+           elog(ERROR, "invalid strategy number %d", opstrategy);
+           break;
+   }
+
+   result->bound_offsets = bms_add_range(NULL, minoff, maxoff);
+   return result;
+}
+
+
+/*
+ * get_matching_range_datums
+ *     Determine the offsets of range bounds matching the specified values,
+ *     according to the semantics of the given operator strategy
+ *
+ * Each datum whose offset is in result is to be treated as the upper bound of
+ * the partition that will contain the desired values.
+ *
+ * If default partition needs to be scanned for given values, set scan_default
+ * in result if present.
+ *
+ * 'opstrategy' if non-zero must be a btree strategy number.
+ *
+ * 'values' contains Datums indexed by the partition key to use for pruning.
+ *
+ * 'nvalues', number of Datums in 'values' array. Must be <= context->partnatts.
+ *
+ * 'partsupfunc' contains the range partitioning comparison functions to be
+ * used to perform partition_range_datum_bsearch or partition_rbound_datum_cmp
+ * using.
+ *
+ * 'nullkeys' is the set of partition keys that are null.
+ */
+static PruneStepResult *
+get_matching_range_bounds(PartitionPruneContext *context,
+                         StrategyNumber opstrategy, Datum *values, int nvalues,
+                         FmgrInfo *partsupfunc, Bitmapset *nullkeys)
+{
+   PruneStepResult *result = (PruneStepResult *) palloc0(sizeof(PruneStepResult));
+   PartitionBoundInfo boundinfo = context->boundinfo;
+   Oid        *partcollation = context->partcollation;
+   int         partnatts = context->partnatts;
+   int        *partindices = boundinfo->indexes;
+   int         off,
+               minoff,
+               maxoff,
+               i;
+   bool        is_equal;
+   bool        inclusive = false;
+
+   Assert(context->strategy == PARTITION_STRATEGY_RANGE);
+   Assert(nvalues <= partnatts);
+
+   result->scan_null = result->scan_default = false;
+
+   /*
+    * If there are no datums to compare keys with, or if we got an IS NULL
+    * clause just return the default partition, if it exists.
+    */
+   if (boundinfo->ndatums == 0 || !bms_is_empty(nullkeys))
+   {
+       result->scan_default = partition_bound_has_default(boundinfo);
+       return result;
+   }
+
+   minoff = 0;
+   maxoff = boundinfo->ndatums;
+
+   /*
+    * If there are no values to compare with the datums in boundinfo, it
+    * means the caller asked for partitions for all non-null datums.  Add
+    * indexes of *all* partitions, including the default partition if one
+    * exists.
+    */
+   if (nvalues == 0)
+   {
+       if (partindices[minoff] < 0)
+           minoff++;
+       if (partindices[maxoff] < 0)
+           maxoff--;
+
+       result->scan_default = partition_bound_has_default(boundinfo);
+       result->bound_offsets = bms_add_range(NULL, minoff, maxoff);
+
+       return result;
+   }
+
+   /*
+    * If the query does not constrain all key columns, we'll need to scan the
+    * the default partition, if any.
+    */
+   if (nvalues < partnatts)
+       result->scan_default = partition_bound_has_default(boundinfo);
+
+   switch (opstrategy)
+   {
+       case BTEqualStrategyNumber:
+           /* Look for the smallest bound that is = lookup value. */
+           off = partition_range_datum_bsearch(partsupfunc,
+                                               partcollation,
+                                               boundinfo,
+                                               nvalues, values,
+                                               &is_equal);
+
+           if (off >= 0 && is_equal)
+           {
+               if (nvalues == partnatts)
+               {
+                   /* There can only be zero or one matching partition. */
+                   if (partindices[off + 1] >= 0)
+                       result->bound_offsets = bms_make_singleton(off + 1);
+                   else
+                       result->scan_default =
+                           partition_bound_has_default(boundinfo);
+                   return result;
+               }
+               else
+               {
+                   int         saved_off = off;
+
+                   /*
+                    * Since the lookup value contains only a prefix of keys,
+                    * we must find other bounds that may also match the
+                    * prefix.  partition_range_datum_bsearch() returns the
+                    * offset of one of them, find others by checking adjacent
+                    * bounds.
+                    */
+
+                   /*
+                    * First find greatest bound that's smaller than the
+                    * lookup value.
+                    */
+                   while (off >= 1)
+                   {
+                       int32       cmpval;
+
+                       cmpval =
+                           partition_rbound_datum_cmp(partsupfunc,
+                                                      partcollation,
+                                                      boundinfo->datums[off - 1],
+                                                      boundinfo->kind[off - 1],
+                                                      values, nvalues);
+                       if (cmpval != 0)
+                           break;
+                       off--;
+                   }
+
+                   Assert(0 ==
+                          partition_rbound_datum_cmp(partsupfunc,
+                                                     partcollation,
+                                                     boundinfo->datums[off],
+                                                     boundinfo->kind[off],
+                                                     values, nvalues));
+
+                   /*
+                    * We can treat 'off' as the offset of the smallest bound
+                    * to be included in the result, if we know it is the
+                    * upper bound of the partition in which the lookup value
+                    * could possibly exist.  One case it couldn't is if the
+                    * bound, or precisely the matched portion of its prefix,
+                    * is not inclusive.
+                    */
+                   if (boundinfo->kind[off][nvalues] ==
+                       PARTITION_RANGE_DATUM_MINVALUE)
+                       off++;
+
+                   minoff = off;
+
+                   /*
+                    * Now find smallest bound that's greater than the lookup
+                    * value.
+                    */
+                   off = saved_off;
+                   while (off < boundinfo->ndatums - 1)
+                   {
+                       int32       cmpval;
+
+                       cmpval = partition_rbound_datum_cmp(partsupfunc,
+                                                           partcollation,
+                                                           boundinfo->datums[off + 1],
+                                                           boundinfo->kind[off + 1],
+                                                           values, nvalues);
+                       if (cmpval != 0)
+                           break;
+                       off++;
+                   }
+
+                   Assert(0 ==
+                          partition_rbound_datum_cmp(partsupfunc,
+                                                     partcollation,
+                                                     boundinfo->datums[off],
+                                                     boundinfo->kind[off],
+                                                     values, nvalues));
+
+                   /*
+                    * off + 1, then would be the offset of the greatest bound
+                    * to be included in the result.
+                    */
+                   maxoff = off + 1;
+               }
+
+               /*
+                * Skip if minoff/maxoff are actually the upper bound of a
+                * un-assigned portion of values.
+                */
+               if (partindices[minoff] < 0 && minoff < boundinfo->ndatums)
+                   minoff++;
+               if (partindices[maxoff] < 0 && maxoff >= 1)
+                   maxoff--;
+
+               /*
+                * There may exist a range of values unassigned to any
+                * non-default partition between the datums at minoff and
+                * maxoff.  Add the default partition in that case.
+                */
+               if (partition_bound_has_default(boundinfo))
+               {
+                   for (i = minoff; i <= maxoff; i++)
+                   {
+                       if (partindices[i] < 0)
+                       {
+                           result->scan_default = true;
+                           break;
+                       }
+                   }
+               }
+
+               Assert(minoff >= 0 && maxoff >= 0);
+               result->bound_offsets = bms_add_range(NULL, minoff, maxoff);
+           }
+           else if (off >= 0)  /* !is_equal */
+           {
+               /*
+                * The lookup value falls in the range between some bounds in
+                * boundinfo.  'off' would be the offset of the greatest bound
+                * that is <= lookup value, so add off + 1 to the result
+                * instead as the offset of the upper bound of the only
+                * partition that may contain the lookup value.
+                */
+               if (partindices[off + 1] >= 0)
+                   result->bound_offsets = bms_make_singleton(off + 1);
+               else
+                   result->scan_default =
+                       partition_bound_has_default(boundinfo);
+           }
+           else
+           {
+               /*
+                * off < 0: the lookup value is smaller than all bounds, so
+                * only the default partition qualifies, if there is one.
+                */
+               result->scan_default = partition_bound_has_default(boundinfo);
+           }
+
+           return result;
+
+       case BTGreaterEqualStrategyNumber:
+           inclusive = true;
+           /* fall through */
+       case BTGreaterStrategyNumber:
+
+           /*
+            * Look for the smallest bound that is > or >= lookup value and
+            * set minoff to its offset.
+            */
+           off = partition_range_datum_bsearch(partsupfunc,
+                                               partcollation,
+                                               boundinfo,
+                                               nvalues, values,
+                                               &is_equal);
+           if (off < 0)
+           {
+               /*
+                * All bounds are greater than the lookup value, so include
+                * all of them in the result.
+                */
+               minoff = 0;
+           }
+           else
+           {
+               if (is_equal && nvalues < partnatts)
+               {
+                   /*
+                    * Since the lookup value contains only a prefix of keys,
+                    * we must find other bounds that may also match the
+                    * prefix.  partition_range_datum_bsearch() returns the
+                    * offset of one of them, find others by checking adjacent
+                    * bounds.
+                    *
+                    * Based on whether the lookup values are inclusive or
+                    * not, we must either include the indexes of all such
+                    * bounds in the result (that is, set minoff to the index
+                    * of smallest such bound) or find the smallest one that's
+                    * greater than the lookup values and set minoff to that.
+                    */
+                   while (off >= 1 && off < boundinfo->ndatums - 1)
+                   {
+                       int32       cmpval;
+                       int         nextoff;
+
+                       nextoff = inclusive ? off - 1 : off + 1;
+                       cmpval =
+                           partition_rbound_datum_cmp(partsupfunc,
+                                                      partcollation,
+                                                      boundinfo->datums[nextoff],
+                                                      boundinfo->kind[nextoff],
+                                                      values, nvalues);
+                       if (cmpval != 0)
+                           break;
+
+                       off = nextoff;
+                   }
+
+                   Assert(0 ==
+                          partition_rbound_datum_cmp(partsupfunc,
+                                                     partcollation,
+                                                     boundinfo->datums[off],
+                                                     boundinfo->kind[off],
+                                                     values, nvalues));
+
+                   minoff = inclusive ? off : off + 1;
+               }
+
+               /*
+                * lookup value falls in the range between some bounds in
+                * boundinfo.  off would be the offset of the greatest bound
+                * that is <= lookup value, so add off + 1 to the result
+                * instead as the offset of the upper bound of the smallest
+                * partition that may contain the lookup value.
+                */
+               else
+                   minoff = off + 1;
+           }
+           break;
+
+       case BTLessEqualStrategyNumber:
+           inclusive = true;
+           /* fall through */
+       case BTLessStrategyNumber:
+
+           /*
+            * Look for the greatest bound that is < or <= lookup value and
+            * set minoff to its offset.
+            */
+           off = partition_range_datum_bsearch(partsupfunc,
+                                               partcollation,
+                                               boundinfo,
+                                               nvalues, values,
+                                               &is_equal);
+           if (off < 0)
+           {
+               /*
+                * All bounds are greater than the key, so we could only
+                * expect to find the lookup key in the default partition.
+                */
+               result->scan_default = partition_bound_has_default(boundinfo);
+               return result;
+           }
+           else
+           {
+               /*
+                * See the comment above.
+                */
+               if (is_equal && nvalues < partnatts)
+               {
+                   while (off >= 1 && off < boundinfo->ndatums - 1)
+                   {
+                       int32       cmpval;
+                       int         nextoff;
+
+                       nextoff = inclusive ? off + 1 : off - 1;
+                       cmpval = partition_rbound_datum_cmp(partsupfunc,
+                                                           partcollation,
+                                                           boundinfo->datums[nextoff],
+                                                           boundinfo->kind[nextoff],
+                                                           values, nvalues);
+                       if (cmpval != 0)
+                           break;
+
+                       off = nextoff;
+                   }
+
+                   Assert(0 ==
+                          partition_rbound_datum_cmp(partsupfunc,
+                                                     partcollation,
+                                                     boundinfo->datums[off],
+                                                     boundinfo->kind[off],
+                                                     values, nvalues));
+
+                   maxoff = inclusive ? off + 1 : off;
+               }
+
+               /*
+                * The lookup value falls in the range between some bounds in
+                * boundinfo.  'off' would be the offset of the greatest bound
+                * that is <= lookup value, so add off + 1 to the result
+                * instead as the offset of the upper bound of the greatest
+                * partition that may contain lookup value.  If the lookup
+                * value had exactly matched the bound, but it isn't
+                * inclusive, no need add the adjacent partition.
+                */
+               else if (!is_equal || inclusive)
+                   maxoff = off + 1;
+               else
+                   maxoff = off;
+           }
+           break;
+
+       default:
+           elog(ERROR, "invalid strategy number %d", opstrategy);
+           break;
+   }
+
+   /*
+    * Skip a gap and when doing so, check if the bound contains a finite
+    * value to decide if we need to add the default partition.  If it's an
+    * infinite bound, we need not add the default partition, as having an
+    * infinite bound means the partition in question catches any values that
+    * would otherwise be in the default partition.
+    */
+   if (partindices[minoff] < 0)
+   {
+       int         lastkey = nvalues - 1;
+
+       if (minoff >= 0 &&
+           minoff < boundinfo->ndatums &&
+           boundinfo->kind[minoff][lastkey] ==
+           PARTITION_RANGE_DATUM_VALUE)
+           result->scan_default = partition_bound_has_default(boundinfo);
+
+       minoff++;
+   }
+
+   /*
+    * Skip a gap.  See the above comment about how we decide whether or or
+    * not to scan the default partition based whether the datum that will
+    * become the maximum datum is finite or not.
+    */
+   if (maxoff >= 1 && partindices[maxoff] < 0)
+   {
+       int         lastkey = nvalues - 1;
+
+       if (maxoff >= 0 &&
+           maxoff <= boundinfo->ndatums &&
+           boundinfo->kind[maxoff - 1][lastkey] ==
+           PARTITION_RANGE_DATUM_VALUE)
+           result->scan_default = partition_bound_has_default(boundinfo);
+
+       maxoff--;
+   }
+
+   if (partition_bound_has_default(boundinfo))
+   {
+       /*
+        * There may exist a range of values unassigned to any non-default
+        * partition between the datums at minoff and maxoff.  Add the default
+        * partition in that case.
+        */
+       for (i = minoff; i <= maxoff; i++)
+       {
+           if (partindices[i] < 0)
+           {
+               result->scan_default = true;
+               break;
+           }
+       }
+   }
+
+   Assert(minoff >= 0 && maxoff >= 0);
+   if (minoff <= maxoff)
+       result->bound_offsets = bms_add_range(NULL, minoff, maxoff);
+
+   return result;
+}
+
+/*
+ * perform_pruning_base_step
+ *     Determines the indexes of datums that satisfy conditions specified in
+ *     'opstep'.
+ *
+ * Result also contains whether special null-accepting and/or default
+ * partition need to be scanned.
+ */
+static PruneStepResult *
+perform_pruning_base_step(PartitionPruneContext *context,
+                         PartitionPruneStepOp *opstep)
+{
+   ListCell   *lc1,
+              *lc2;
+   int         keyno,
+               nvalues;
+   Datum       values[PARTITION_MAX_KEYS];
+   FmgrInfo    partsupfunc[PARTITION_MAX_KEYS];
+
+   /*
+    * There better be the same number of expressions and compare functions.
+    */
+   Assert(list_length(opstep->exprs) == list_length(opstep->cmpfns));
+
+   nvalues = 0;
+   lc1 = list_head(opstep->exprs);
+   lc2 = list_head(opstep->cmpfns);
+
+   /*
+    * Generate the partition lookup key that will be used by one of the
+    * get_matching_*_bounds functions called below.
+    */
+   for (keyno = 0; keyno < context->partnatts; keyno++)
+   {
+       /*
+        * For hash partitioning, it is possible that values of some keys are
+        * not provided in operator clauses, but instead the planner found
+        * that they appeared in a IS NULL clause.
+        */
+       if (bms_is_member(keyno, opstep->nullkeys))
+           continue;
+
+       /*
+        * For range partitioning, we must only perform pruning with values
+        * for either all partition keys or a prefix thereof.
+        */
+       if (keyno > nvalues && context->strategy == PARTITION_STRATEGY_RANGE)
+           break;
+
+       if (lc1 != NULL)
+       {
+           Expr       *expr;
+           Datum       datum;
+
+           expr = lfirst(lc1);
+           if (partkey_datum_from_expr(context, expr, &datum))
+           {
+               Oid         cmpfn;
+
+               /*
+                * If we're going to need a different comparison function than
+                * the one cached in the PartitionKey, we'll need to look up
+                * the FmgrInfo.
+                */
+               cmpfn = lfirst_oid(lc2);
+               Assert(OidIsValid(cmpfn));
+               if (cmpfn != context->partsupfunc[keyno].fn_oid)
+                   fmgr_info(cmpfn, &partsupfunc[keyno]);
+               else
+                   fmgr_info_copy(&partsupfunc[keyno],
+                                  &context->partsupfunc[keyno],
+                                  CurrentMemoryContext);
+
+               values[keyno] = datum;
+               nvalues++;
+           }
+
+           lc1 = lnext(lc1);
+           lc2 = lnext(lc2);
+       }
+   }
+
+   switch (context->strategy)
+   {
+       case PARTITION_STRATEGY_HASH:
+           return get_matching_hash_bounds(context,
+                                           opstep->opstrategy,
+                                           values, nvalues,
+                                           partsupfunc,
+                                           opstep->nullkeys);
+
+       case PARTITION_STRATEGY_LIST:
+           return get_matching_list_bounds(context,
+                                           opstep->opstrategy,
+                                           values[0], nvalues,
+                                           &partsupfunc[0],
+                                           opstep->nullkeys);
+
+       case PARTITION_STRATEGY_RANGE:
+           return get_matching_range_bounds(context,
+                                            opstep->opstrategy,
+                                            values, nvalues,
+                                            partsupfunc,
+                                            opstep->nullkeys);
+
+       default:
+           elog(ERROR, "unexpected partition strategy: %d",
+                (int) context->strategy);
+           break;
+   }
+
+   return NULL;
+}
+
+/*
+ * perform_pruning_combine_step
+ *     Determines the indexes of datums obtained by combining those given
+ *     by the steps identified by cstep->source_stepids using the specified
+ *     combination method
+ *
+ * Since cstep may refer to the result of earlier steps, we also receive
+ * step_results here.
+ */
+static PruneStepResult *
+perform_pruning_combine_step(PartitionPruneContext *context,
+                            PartitionPruneStepCombine *cstep,
+                            PruneStepResult **step_results)
+{
+   ListCell   *lc1;
+   PruneStepResult *result = NULL;
+   bool        firststep;
+
+   /*
+    * A combine step without any source steps is an indication to not perform
+    * any partition pruning, we just return all partitions.
+    */
+   result = (PruneStepResult *) palloc0(sizeof(PruneStepResult));
+   if (list_length(cstep->source_stepids) == 0)
+   {
+       PartitionBoundInfo boundinfo = context->boundinfo;
+
+       result->bound_offsets = bms_add_range(NULL, 0, boundinfo->ndatums - 1);
+       result->scan_default = partition_bound_has_default(boundinfo);
+       result->scan_null = partition_bound_accepts_nulls(boundinfo);
+       return result;
+   }
+
+   switch (cstep->combineOp)
+   {
+       case PARTPRUNE_COMBINE_UNION:
+           foreach(lc1, cstep->source_stepids)
+           {
+               int         step_id = lfirst_int(lc1);
+               PruneStepResult *step_result;
+
+               /*
+                * step_results[step_id] must contain a valid result, which is
+                * confirmed by the fact that cstep's step_id is greater than
+                * step_id and the fact that results of the individual steps
+                * are evaluated in sequence of their step_ids.
+                */
+               if (step_id >= cstep->step.step_id)
+                   elog(ERROR, "invalid pruning combine step argument");
+               step_result = step_results[step_id];
+               Assert(step_result != NULL);
+
+               /* Record any additional datum indexes from this step */
+               result->bound_offsets = bms_add_members(result->bound_offsets,
+                                                       step_result->bound_offsets);
+
+               /* Update whether to scan null and default partitions. */
+               if (!result->scan_null)
+                   result->scan_null = step_result->scan_null;
+               if (!result->scan_default)
+                   result->scan_default = step_result->scan_default;
+           }
+           break;
+
+       case PARTPRUNE_COMBINE_INTERSECT:
+           firststep = true;
+           foreach(lc1, cstep->source_stepids)
+           {
+               int         step_id = lfirst_int(lc1);
+               PruneStepResult *step_result;
+
+               if (step_id >= cstep->step.step_id)
+                   elog(ERROR, "invalid pruning combine step argument");
+               step_result = step_results[step_id];
+               Assert(step_result != NULL);
+
+               if (firststep)
+               {
+                   /* Copy step's result the first time. */
+                   result->bound_offsets = step_result->bound_offsets;
+                   result->scan_null = step_result->scan_null;
+                   result->scan_default = step_result->scan_default;
+                   firststep = false;
+               }
+               else
+               {
+                   /* Record datum indexes common to both steps */
+                   result->bound_offsets =
+                       bms_int_members(result->bound_offsets,
+                                       step_result->bound_offsets);
+
+                   /* Update whether to scan null and default partitions. */
+                   if (result->scan_null)
+                       result->scan_null = step_result->scan_null;
+                   if (result->scan_default)
+                       result->scan_default = step_result->scan_default;
+               }
+           }
+           break;
+
+       default:
+           elog(ERROR, "invalid pruning combine op: %d",
+                (int) cstep->combineOp);
+   }
+
+   return result;
+}
+
+/*
+ * match_boolean_partition_clause
+ *
+ * Sets *outconst to a Const containing true or false value and returns true if
+ * we're able to match the clause to the partition key as specially-shaped
+ * Boolean clause.  Returns false otherwise with *outconst set to NULL.
+ */
+static bool
+match_boolean_partition_clause(Oid partopfamily, Expr *clause, Expr *partkey,
+                              Expr **outconst)
+{
+   Expr       *leftop;
+
+   *outconst = NULL;
+
+   if (!IsBooleanOpfamily(partopfamily))
+       return false;
+
+   if (IsA(clause, BooleanTest))
+   {
+       BooleanTest *btest = (BooleanTest *) clause;
+
+       /* Only IS [NOT] TRUE/FALSE are any good to us */
+       if (btest->booltesttype == IS_UNKNOWN ||
+           btest->booltesttype == IS_NOT_UNKNOWN)
+           return false;
+
+       leftop = btest->arg;
+       if (IsA(leftop, RelabelType))
+           leftop = ((RelabelType *) leftop)->arg;
+
+       if (equal(leftop, partkey))
+           *outconst = (btest->booltesttype == IS_TRUE ||
+                        btest->booltesttype == IS_NOT_FALSE)
+               ? (Expr *) makeBoolConst(true, false)
+               : (Expr *) makeBoolConst(false, false);
+
+       if (*outconst)
+           return true;
+   }
+   else
+   {
+       bool        is_not_clause = not_clause((Node *) clause);
+
+       leftop = is_not_clause ? get_notclausearg(clause) : clause;
+
+       if (IsA(leftop, RelabelType))
+           leftop = ((RelabelType *) leftop)->arg;
+
+       /* Compare to the partition key, and make up a clause ... */
+       if (equal(leftop, partkey))
+           *outconst = is_not_clause ?
+               (Expr *) makeBoolConst(false, false) :
+               (Expr *) makeBoolConst(true, false);
+       else if (equal(negate_clause((Node *) leftop), partkey))
+           *outconst = (Expr *) makeBoolConst(false, false);
+
+       if (*outconst)
+           return true;
+   }
+
+   return false;
+}
+
+/*
+ * partkey_datum_from_expr
+ *     Evaluate 'expr', 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)
+{
+   switch (nodeTag(expr))
+   {
+       case T_Const:
+           *value = ((Const *) expr)->constvalue;
+           return true;
+
+       default:
+           break;
+   }
+
+   return false;
+}
index cbeca5776bd5525e6ae62190dc2abbc2d7013d17..7a0c5d36db3b6d24bf69cbd0fe95445e61318552 100644 (file)
@@ -53,6 +53,6 @@
  */
 
 /*                         yyyymmddN */
-#define CATALOG_VERSION_NO 201804052
+#define CATALOG_VERSION_NO 201804061
 
 #endif
index cd15faa7a14463961c49f933872b621ed35b2896..b25e25bf9dfc4e7970ec6b4871bc0f4cf840e2df 100644 (file)
@@ -26,7 +26,7 @@
  * PartitionBoundInfo encapsulates a set of partition bounds.  It is usually
  * associated with partitioned tables as part of its partition descriptor.
  *
- * The internal structure is opaque outside partition.c.
+ * The internal structure appears in partbounds.h.
  */
 typedef struct PartitionBoundInfoData *PartitionBoundInfo;
 
@@ -70,7 +70,6 @@ extern void check_default_allows_bound(Relation parent, Relation defaultRel,
                           PartitionBoundSpec *new_spec);
 extern List *get_proposed_default_constraint(List *new_part_constaints);
 
-/* For tuple routing */
 extern int get_partition_for_tuple(Relation relation, Datum *values,
                        bool *isnull);
 
index b5444742548bee4a02f42676c8d9b1734c99b191..5b20dd77a17a2c743f05d7f2f4576ac1c5ef1685 100644 (file)
@@ -53,6 +53,9 @@ typedef FormData_pg_opfamily *Form_pg_opfamily;
 #define Anum_pg_opfamily_opfnamespace  3
 #define Anum_pg_opfamily_opfowner      4
 
+#define IsBooleanOpfamily(opfamily) \
+   ((opfamily) == BOOL_BTREE_FAM_OID || (opfamily) == BOOL_HASH_FAM_OID)
+
 /* ----------------
  *     initial contents of pg_opfamily
  * ----------------
index b1e3d53f78f8d327396b83eb6304ab97272c172b..4fc2de7184ac67e98521c675c56fbc1777b42c05 100644 (file)
@@ -193,6 +193,9 @@ typedef enum NodeTag
    T_FromExpr,
    T_OnConflictExpr,
    T_IntoClause,
+   T_PartitionPruneStep,
+   T_PartitionPruneStepOp,
+   T_PartitionPruneStepCombine,
 
    /*
     * TAGS FOR EXPRESSION STATE NODES (execnodes.h)
@@ -262,7 +265,6 @@ typedef enum NodeTag
    T_PlaceHolderVar,
    T_SpecialJoinInfo,
    T_AppendRelInfo,
-   T_PartitionedChildRelInfo,
    T_PlaceHolderInfo,
    T_MinMaxAggInfo,
    T_PlannerParamItem,
index 1b4b0d75afa042a44930773cfe66ea347f1c69ef..ff5c4ff8e4a82a4712ca75a14bc3d551f0f37c1f 100644 (file)
@@ -18,6 +18,7 @@
 #define PRIMNODES_H
 
 #include "access/attnum.h"
+#include "access/stratnum.h"
 #include "nodes/bitmapset.h"
 #include "nodes/pg_list.h"
 
@@ -1506,4 +1507,78 @@ typedef struct OnConflictExpr
    List       *exclRelTlist;   /* tlist of the EXCLUDED pseudo relation */
 } OnConflictExpr;
 
+
+/*
+ * Node types to represent a partition pruning step.
+ */
+
+/*
+ * The base Node type.  step_id is the global identifier of a given step
+ * within a given pruning context.
+ */
+typedef struct PartitionPruneStep
+{
+   NodeTag     type;
+   int         step_id;
+} PartitionPruneStep;
+
+/*----------
+ * PartitionPruneStepOp - Information to prune using a set of mutually AND'd
+ *                         OpExpr clauses
+ *
+ * This contains information extracted from up to partnatts OpExpr clauses,
+ * where partnatts is the number of partition key columns.  'opstrategy' is the
+ * strategy of the operator in the clause matched to the last partition key.
+ * 'exprs' contains expressions which comprise the lookup key to be passed to
+ * the partition bound search function.  'cmpfns' contains the OIDs of
+ * comparison function used to compare aforementioned expressions with
+ * partition bounds.  Both 'exprs' and 'cmpfns' contain the same number of
+ * items up to partnatts items.
+ *
+ * Once we find the offset of a partition bound using the lookup key, we
+ * determine which partitions to include in the result based on the value of
+ * 'opstrategy'.  For example, if it were equality, we'd return just the
+ * partition that would contain that key or a set of partitions if the key
+ * didn't consist of all partitioning columns.  For non-equality strategies,
+ * we'd need to include other partitions as appropriate.
+ *
+ * 'nullkeys' is the set containing the offset of the partition keys (0 to
+ * partnatts - 1) that were matched to an IS NULL clause.  This is only
+ * considered for hash partitioning as we need to pass which keys are null
+ * to the hash partition bound search function.  It is never possible to
+ * have an expression be present in 'exprs' for a given partition key and
+ * the corresponding bit set in 'nullkeys'.
+ *----------
+ */
+typedef struct PartitionPruneStepOp
+{
+   PartitionPruneStep step;
+
+   StrategyNumber opstrategy;
+   List       *exprs;
+   List       *cmpfns;
+   Bitmapset  *nullkeys;
+} PartitionPruneStepOp;
+
+/*----------
+ * PartitionPruneStepCombine - Information to prune using a BoolExpr clause
+ *
+ * For BoolExpr clauses, we combine the set of partitions determined for each
+ * of its argument clauses.
+ *----------
+ */
+typedef enum PartitionPruneCombineOp
+{
+   PARTPRUNE_COMBINE_UNION,
+   PARTPRUNE_COMBINE_INTERSECT
+} PartitionPruneCombineOp;
+
+typedef struct PartitionPruneStepCombine
+{
+   PartitionPruneStep step;
+
+   PartitionPruneCombineOp combineOp;
+   List       *source_stepids;
+} PartitionPruneStepCombine;
+
 #endif                         /* PRIMNODES_H */
index a2dde70de58742cffad1647138f582459f704bf0..acb8814924f3ec29d728a8c03e26fefd881ddc7e 100644 (file)
@@ -15,6 +15,7 @@
 #define RELATION_H
 
 #include "access/sdir.h"
+#include "fmgr.h"
 #include "lib/stringinfo.h"
 #include "nodes/params.h"
 #include "nodes/parsenodes.h"
@@ -253,8 +254,6 @@ typedef struct PlannerInfo
 
    List       *append_rel_list;    /* list of AppendRelInfos */
 
-   List       *pcinfo_list;    /* list of PartitionedChildRelInfos */
-
    List       *rowMarks;       /* list of PlanRowMarks */
 
    List       *placeholder_list;   /* list of PlaceHolderInfos */
@@ -319,6 +318,9 @@ typedef struct PlannerInfo
 
    /* optional private data for join_search_hook, e.g., GEQO */
    void       *join_search_private;
+
+   /* Does this query modify any partition key columns? */
+   bool        partColsUpdated;
 } PlannerInfo;
 
 
@@ -356,6 +358,9 @@ typedef struct PartitionSchemeData
    /* Cached information about partition key data types. */
    int16      *parttyplen;
    bool       *parttypbyval;
+
+   /* Cached information about partition comparison functions. */
+   FmgrInfo   *partsupfunc;
 }          PartitionSchemeData;
 
 typedef struct PartitionSchemeData *PartitionScheme;
@@ -528,11 +533,15 @@ typedef struct PartitionSchemeData *PartitionScheme;
  *
  * If the relation is partitioned, these fields will be set:
  *
- *         part_scheme - Partitioning scheme of the relation
- *         boundinfo - Partition bounds
- *         nparts - Number of partitions
- *         part_rels - RelOptInfos for each partition
- *         partexprs, nullable_partexprs - Partition key expressions
+ *     part_scheme - Partitioning scheme of the relation
+ *     nparts - Number of partitions
+ *     boundinfo - Partition bounds
+ *     partition_qual - Partition constraint if not the root
+ *     part_rels - RelOptInfos for each partition
+ *     partexprs, nullable_partexprs - Partition key expressions
+ *     partitioned_child_rels - RT indexes of unpruned partitions of
+ *                              relation that are partitioned tables
+ *                              themselves
  *
  * Note: A base relation always has only one set of partition keys, but a join
  * relation may have as many sets of partition keys as the number of relations
@@ -663,10 +672,12 @@ typedef struct RelOptInfo
    PartitionScheme part_scheme;    /* Partitioning scheme. */
    int         nparts;         /* number of partitions */
    struct PartitionBoundInfoData *boundinfo;   /* Partition bounds */
+   List       *partition_qual; /* partition constraint */
    struct RelOptInfo **part_rels;  /* Array of RelOptInfos of partitions,
                                     * stored in the same order of bounds */
    List      **partexprs;      /* Non-nullable partition key expressions. */
    List      **nullable_partexprs; /* Nullable partition key expressions. */
+   List       *partitioned_child_rels; /* List of RT indexes. */
 } RelOptInfo;
 
 /*
@@ -1686,7 +1697,7 @@ typedef struct ModifyTablePath
    List       *partitioned_rels;
    bool        partColsUpdated;    /* some part key in hierarchy updated */
    List       *resultRelations;    /* integer list of RT indexes */
-   Index       mergeTargetRelation;/* RT index of merge target relation */
+   Index       mergeTargetRelation;    /* RT index of merge target relation */
    List       *subpaths;       /* Path(s) producing source data */
    List       *subroots;       /* per-target-table PlannerInfos */
    List       *withCheckOptionLists;   /* per-target-table WCO lists */
@@ -2121,27 +2132,6 @@ typedef struct AppendRelInfo
    Oid         parent_reloid;  /* OID of parent relation */
 } AppendRelInfo;
 
-/*
- * For a partitioned table, this maps its RT index to the list of RT indexes
- * of the partitioned child tables in the partition tree.  We need to
- * separately store this information, because we do not create AppendRelInfos
- * for the partitioned child tables of a parent table, since AppendRelInfos
- * contain information that is unnecessary for the partitioned child tables.
- * The child_rels list must contain at least one element, because the parent
- * partitioned table is itself counted as a child.
- *
- * These structs are kept in the PlannerInfo node's pcinfo_list.
- */
-typedef struct PartitionedChildRelInfo
-{
-   NodeTag     type;
-
-   Index       parent_relid;
-   List       *child_rels;
-   bool        part_cols_updated;  /* is the partition key of any of
-                                    * the partitioned tables updated? */
-} PartitionedChildRelInfo;
-
 /*
  * For each distinct placeholder expression generated during planning, we
  * store a PlaceHolderInfo node in the PlannerInfo node's placeholder_list.
index 07a3bc0627cb8458d120b5eef06c4a7898cb0154..c090396e139aa31cf448548a40a2ba889ec63239 100644 (file)
@@ -59,9 +59,4 @@ extern Expr *preprocess_phv_expression(PlannerInfo *root, Expr *expr);
 extern bool plan_cluster_use_sort(Oid tableOid, Oid indexOid);
 extern int plan_create_index_workers(Oid tableOid, Oid indexOid);
 
-extern List *get_partitioned_child_rels(PlannerInfo *root, Index rti,
-                          bool *part_cols_updated);
-extern List *get_partitioned_child_rels_for_join(PlannerInfo *root,
-                                   Relids join_relids);
-
 #endif                         /* PLANNER_H */
diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h
new file mode 100644 (file)
index 0000000..c76014d
--- /dev/null
@@ -0,0 +1,124 @@
+/*-------------------------------------------------------------------------
+ *
+ * partbounds.h
+ *
+ * Copyright (c) 2007-2018, PostgreSQL Global Development Group
+ *
+ * src/include/partitioning/partbounds.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTBOUNDS_H
+#define PARTBOUNDS_H
+
+#include "catalog/partition.h"
+
+
+/*
+ * PartitionBoundInfoData encapsulates a set of partition bounds. It is
+ * usually associated with partitioned tables as part of its partition
+ * descriptor, but may also be used to represent a virtual partitioned
+ * table such as a partitioned joinrel within the planner.
+ *
+ * A list partition datum that is known to be NULL is never put into the
+ * datums array. Instead, it is tracked using the null_index field.
+ *
+ * In the case of range partitioning, ndatums will typically be far less than
+ * 2 * nparts, because a partition's upper bound and the next partition's lower
+ * bound are the same in most common cases, and we only store one of them (the
+ * upper bound).  In case of hash partitioning, ndatums will be same as the
+ * number of partitions.
+ *
+ * For range and list partitioned tables, datums is an array of datum-tuples
+ * with key->partnatts datums each.  For hash partitioned tables, it is an array
+ * of datum-tuples with 2 datums, modulus and remainder, corresponding to a
+ * given partition.
+ *
+ * The datums in datums array are arranged in increasing order as defined by
+ * functions qsort_partition_rbound_cmp(), qsort_partition_list_value_cmp() and
+ * qsort_partition_hbound_cmp() for range, list and hash partitioned tables
+ * respectively. For range and list partitions this simply means that the
+ * datums in the datums array are arranged in increasing order as defined by
+ * the partition key's operator classes and collations.
+ *
+ * In the case of list partitioning, the indexes array stores one entry for
+ * every datum, which is the index of the partition that accepts a given datum.
+ * In case of range partitioning, it stores one entry per distinct range
+ * datum, which is the index of the partition for which a given datum
+ * is an upper bound.  In the case of hash partitioning, the number of the
+ * entries in the indexes array is same as the greatest modulus amongst all
+ * partitions.  For a given partition key datum-tuple, the index of the
+ * partition which would accept that datum-tuple would be given by the entry
+ * pointed by remainder produced when hash value of the datum-tuple is divided
+ * by the greatest modulus.
+ */
+
+typedef struct PartitionBoundInfoData
+{
+   char        strategy;       /* hash, list or range? */
+   int         ndatums;        /* Length of the datums following array */
+   Datum     **datums;
+   PartitionRangeDatumKind **kind; /* The kind of each range bound datum;
+                                    * NULL for hash and list partitioned
+                                    * tables */
+   int        *indexes;        /* Partition indexes */
+   int         null_index;     /* Index of the null-accepting partition; -1
+                                * if there isn't one */
+   int         default_index;  /* Index of the default partition; -1 if there
+                                * isn't one */
+} PartitionBoundInfoData;
+
+#define partition_bound_accepts_nulls(bi) ((bi)->null_index != -1)
+#define partition_bound_has_default(bi) ((bi)->default_index != -1)
+
+/*
+ * When qsort'ing partition bounds after reading from the catalog, each bound
+ * is represented with one of the following structs.
+ */
+
+/* One bound of a hash partition */
+typedef struct PartitionHashBound
+{
+   int         modulus;
+   int         remainder;
+   int         index;
+} PartitionHashBound;
+
+/* One value coming from some (index'th) list partition */
+typedef struct PartitionListValue
+{
+   int         index;
+   Datum       value;
+} PartitionListValue;
+
+/* One bound of a range partition */
+typedef struct PartitionRangeBound
+{
+   int         index;
+   Datum      *datums;         /* range bound datums */
+   PartitionRangeDatumKind *kind;  /* the kind of each datum */
+   bool        lower;          /* this is the lower (vs upper) bound */
+} PartitionRangeBound;
+
+extern int get_hash_partition_greatest_modulus(PartitionBoundInfo b);
+extern int partition_list_bsearch(FmgrInfo *partsupfunc, Oid *partcollation,
+                      PartitionBoundInfo boundinfo,
+                      Datum value, bool *is_equal);
+extern int partition_range_bsearch(int partnatts, FmgrInfo *partsupfunc,
+                       Oid *partcollation,
+                       PartitionBoundInfo boundinfo,
+                       PartitionRangeBound *probe, bool *is_equal);
+extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc,
+                             Oid *partcollation,
+                             PartitionBoundInfo boundinfo,
+                             int nvalues, Datum *values, bool *is_equal);
+extern int partition_hash_bsearch(PartitionBoundInfo boundinfo,
+                      int modulus, int remainder);
+extern uint64 compute_hash_value(int partnatts, FmgrInfo *partsupfunc,
+                  Datum *values, bool *isnull);
+extern int32 partition_rbound_datum_cmp(FmgrInfo *partsupfunc,
+                          Oid *partcollation,
+                          Datum *rb_datums, PartitionRangeDatumKind *rb_kind,
+                          Datum *tuple_datums, int n_tuple_datums);
+
+#endif                         /* PARTBOUNDS_H */
diff --git a/src/include/partitioning/partprune.h b/src/include/partitioning/partprune.h
new file mode 100644 (file)
index 0000000..52fadc7
--- /dev/null
@@ -0,0 +1,49 @@
+/*-------------------------------------------------------------------------
+ *
+ * partprune.h
+ *   prototypes for partprune.c
+ *
+ *
+ * Portions Copyright (c) 1996-2018, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/partitioning/partprune.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef PARTPRUNE_H
+#define PARTPRUNE_H
+
+#include "catalog/partition.h"
+#include "nodes/relation.h"
+
+/*
+ * PartitionPruneContext
+ *
+ * Information about a partitioned table needed to perform partition pruning.
+ */
+typedef struct PartitionPruneContext
+{
+   /* Partition key information */
+   char        strategy;
+   int         partnatts;
+   Oid        *partopfamily;
+   Oid        *partopcintype;
+   Oid        *partcollation;
+   FmgrInfo   *partsupfunc;
+
+   /* Number of partitions */
+   int         nparts;
+
+   /* Partition boundary info */
+   PartitionBoundInfo boundinfo;
+} PartitionPruneContext;
+
+
+extern Relids prune_append_rel_partitions(RelOptInfo *rel);
+extern Bitmapset *get_matching_partitions(PartitionPruneContext *context,
+                       List *pruning_steps);
+extern List *gen_partprune_steps(RelOptInfo *rel, List *clauses,
+                   bool *contradictory);
+
+#endif                         /* PARTPRUNE_H */
index 5e57b9a4658b60432f989f1d38b700bfb857da40..b2b912ed5c19e866f70b752d0b49d1211e983a87 100644 (file)
@@ -1951,11 +1951,13 @@ explain (costs off) select * from mcrparted where abs(b) = 5;   -- scans all parti
          Filter: (abs(b) = 5)
    ->  Seq Scan on mcrparted3
          Filter: (abs(b) = 5)
+   ->  Seq Scan on mcrparted4
+         Filter: (abs(b) = 5)
    ->  Seq Scan on mcrparted5
          Filter: (abs(b) = 5)
    ->  Seq Scan on mcrparted_def
          Filter: (abs(b) = 5)
-(13 rows)
+(15 rows)
 
 explain (costs off) select * from mcrparted where a > -1;  -- scans all partitions
              QUERY PLAN              
index 09517775b66172a3ccc9590c0f2c18f351eb83c0..2d77b3edd4afe6d3ce8db73ad3ce7a4133231a52 100644 (file)
@@ -208,16 +208,14 @@ explain (costs off) select * from rlp where 1 > a;    /* commuted */
 (3 rows)
 
 explain (costs off) select * from rlp where a <= 1;
-              QUERY PLAN               
----------------------------------------
+        QUERY PLAN        
+--------------------------
  Append
    ->  Seq Scan on rlp1
          Filter: (a <= 1)
    ->  Seq Scan on rlp2
          Filter: (a <= 1)
-   ->  Seq Scan on rlp_default_default
-         Filter: (a <= 1)
-(7 rows)
+(5 rows)
 
 explain (costs off) select * from rlp where a = 1;
        QUERY PLAN        
@@ -235,7 +233,7 @@ explain (costs off) select * from rlp where a = 1::bigint;      /* same as above */
          Filter: (a = '1'::bigint)
 (3 rows)
 
-explain (costs off) select * from rlp where a = 1::numeric;    /* only null can be pruned */
+explain (costs off) select * from rlp where a = 1::numeric;        /* no pruning */
                   QUERY PLAN                   
 -----------------------------------------------
  Append
@@ -265,9 +263,11 @@ explain (costs off) select * from rlp where a = 1::numeric;    /* only null can be
          Filter: ((a)::numeric = '1'::numeric)
    ->  Seq Scan on rlp_default_30
          Filter: ((a)::numeric = '1'::numeric)
+   ->  Seq Scan on rlp_default_null
+         Filter: ((a)::numeric = '1'::numeric)
    ->  Seq Scan on rlp_default_default
          Filter: ((a)::numeric = '1'::numeric)
-(29 rows)
+(31 rows)
 
 explain (costs off) select * from rlp where a <= 10;
               QUERY PLAN               
@@ -575,7 +575,9 @@ explain (costs off) select * from rlp where a > 20 and a < 27;
          Filter: ((a > 20) AND (a < 27))
    ->  Seq Scan on rlp4_default
          Filter: ((a > 20) AND (a < 27))
-(7 rows)
+   ->  Seq Scan on rlp_default_default
+         Filter: ((a > 20) AND (a < 27))
+(9 rows)
 
 explain (costs off) select * from rlp where a = 29;
            QUERY PLAN           
@@ -714,9 +716,7 @@ explain (costs off) select * from mc3p where a = 1 and abs(b) = 1 and c < 8;
          Filter: ((c < 8) AND (a = 1) AND (abs(b) = 1))
    ->  Seq Scan on mc3p1
          Filter: ((c < 8) AND (a = 1) AND (abs(b) = 1))
-   ->  Seq Scan on mc3p_default
-         Filter: ((c < 8) AND (a = 1) AND (abs(b) = 1))
-(7 rows)
+(5 rows)
 
 explain (costs off) select * from mc3p where a = 10 and abs(b) between 5 and 35;
                            QUERY PLAN                            
@@ -892,6 +892,8 @@ explain (costs off) select * from mc3p where a = 1 or abs(b) = 1 or c = 1;
          Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
    ->  Seq Scan on mc3p2
          Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
+   ->  Seq Scan on mc3p3
+         Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
    ->  Seq Scan on mc3p4
          Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
    ->  Seq Scan on mc3p5
@@ -902,7 +904,7 @@ explain (costs off) select * from mc3p where a = 1 or abs(b) = 1 or c = 1;
          Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
    ->  Seq Scan on mc3p_default
          Filter: ((a = 1) OR (abs(b) = 1) OR (c = 1))
-(17 rows)
+(19 rows)
 
 explain (costs off) select * from mc3p where (a = 1 and abs(b) = 1) or (a = 10 and abs(b) = 10);
                                   QUERY PLAN                                  
@@ -1007,24 +1009,20 @@ explain (costs off) select * from boolpart where a in (true, false);
 (5 rows)
 
 explain (costs off) select * from boolpart where a = false;
-             QUERY PLAN             
-------------------------------------
+          QUERY PLAN          
+------------------------------
  Append
    ->  Seq Scan on boolpart_f
          Filter: (NOT a)
-   ->  Seq Scan on boolpart_default
-         Filter: (NOT a)
-(5 rows)
+(3 rows)
 
 explain (costs off) select * from boolpart where not a = false;
-             QUERY PLAN             
-------------------------------------
+          QUERY PLAN          
+------------------------------
  Append
    ->  Seq Scan on boolpart_t
          Filter: a
-   ->  Seq Scan on boolpart_default
-         Filter: a
-(5 rows)
+(3 rows)
 
 explain (costs off) select * from boolpart where a is true or a is not true;
                     QUERY PLAN                    
@@ -1034,33 +1032,22 @@ explain (costs off) select * from boolpart where a is true or a is not true;
          Filter: ((a IS TRUE) OR (a IS NOT TRUE))
    ->  Seq Scan on boolpart_t
          Filter: ((a IS TRUE) OR (a IS NOT TRUE))
-   ->  Seq Scan on boolpart_default
-         Filter: ((a IS TRUE) OR (a IS NOT TRUE))
-(7 rows)
+(5 rows)
 
 explain (costs off) select * from boolpart where a is not true;
-             QUERY PLAN             
-------------------------------------
+           QUERY PLAN            
+---------------------------------
  Append
    ->  Seq Scan on boolpart_f
          Filter: (a IS NOT TRUE)
-   ->  Seq Scan on boolpart_t
-         Filter: (a IS NOT TRUE)
-   ->  Seq Scan on boolpart_default
-         Filter: (a IS NOT TRUE)
-(7 rows)
+(3 rows)
 
 explain (costs off) select * from boolpart where a is not true and a is not false;
-                       QUERY PLAN                       
---------------------------------------------------------
- Append
-   ->  Seq Scan on boolpart_f
-         Filter: ((a IS NOT TRUE) AND (a IS NOT FALSE))
-   ->  Seq Scan on boolpart_t
-         Filter: ((a IS NOT TRUE) AND (a IS NOT FALSE))
-   ->  Seq Scan on boolpart_default
-         Filter: ((a IS NOT TRUE) AND (a IS NOT FALSE))
-(7 rows)
+        QUERY PLAN        
+--------------------------
+ Result
+   One-Time Filter: false
+(2 rows)
 
 explain (costs off) select * from boolpart where a is unknown;
              QUERY PLAN             
@@ -1086,4 +1073,446 @@ explain (costs off) select * from boolpart where a is not unknown;
          Filter: (a IS NOT UNKNOWN)
 (7 rows)
 
-drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart;
+--
+-- some more cases
+--
+--
+-- pruning for partitioned table appearing inside a sub-query
+--
+-- pruning won't work for mc3p, because some keys are Params
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.a = t1.b and abs(t2.b) = 1 and t2.c = 1) s where t1.a = 1;
+                              QUERY PLAN                               
+-----------------------------------------------------------------------
+ Nested Loop
+   ->  Append
+         ->  Seq Scan on mc2p1 t1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p2 t1_1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p_default t1_2
+               Filter: (a = 1)
+   ->  Aggregate
+         ->  Append
+               ->  Seq Scan on mc3p0 t2
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p1 t2_1
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p2 t2_2
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p3 t2_3
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p4 t2_4
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p5 t2_5
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p6 t2_6
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p7 t2_7
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p_default t2_8
+                     Filter: ((a = t1.b) AND (c = 1) AND (abs(b) = 1))
+(28 rows)
+
+-- pruning should work fine, because values for a prefix of keys (a, b) are
+-- available
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.c = t1.b and abs(t2.b) = 1 and t2.a = 1) s where t1.a = 1;
+                              QUERY PLAN                               
+-----------------------------------------------------------------------
+ Nested Loop
+   ->  Append
+         ->  Seq Scan on mc2p1 t1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p2 t1_1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p_default t1_2
+               Filter: (a = 1)
+   ->  Aggregate
+         ->  Append
+               ->  Seq Scan on mc3p0 t2
+                     Filter: ((c = t1.b) AND (a = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p1 t2_1
+                     Filter: ((c = t1.b) AND (a = 1) AND (abs(b) = 1))
+               ->  Seq Scan on mc3p_default t2_2
+                     Filter: ((c = t1.b) AND (a = 1) AND (abs(b) = 1))
+(16 rows)
+
+-- also here, because values for all keys are provided
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.a = 1 and abs(t2.b) = 1 and t2.c = 1) s where t1.a = 1;
+                             QUERY PLAN                             
+--------------------------------------------------------------------
+ Nested Loop
+   ->  Aggregate
+         ->  Append
+               ->  Seq Scan on mc3p1 t2
+                     Filter: ((a = 1) AND (c = 1) AND (abs(b) = 1))
+   ->  Append
+         ->  Seq Scan on mc2p1 t1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p2 t1_1
+               Filter: (a = 1)
+         ->  Seq Scan on mc2p_default t1_2
+               Filter: (a = 1)
+(12 rows)
+
+--
+-- pruning with clauses containing <> operator
+--
+-- doesn't prune range partitions
+create table rp (a int) partition by range (a);
+create table rp0 partition of rp for values from (minvalue) to (1);
+create table rp1 partition of rp for values from (1) to (2);
+create table rp2 partition of rp for values from (2) to (maxvalue);
+explain (costs off) select * from rp where a <> 1;
+        QUERY PLAN        
+--------------------------
+ Append
+   ->  Seq Scan on rp0
+         Filter: (a <> 1)
+   ->  Seq Scan on rp1
+         Filter: (a <> 1)
+   ->  Seq Scan on rp2
+         Filter: (a <> 1)
+(7 rows)
+
+explain (costs off) select * from rp where a <> 1 and a <> 2;
+               QUERY PLAN                
+-----------------------------------------
+ Append
+   ->  Seq Scan on rp0
+         Filter: ((a <> 1) AND (a <> 2))
+   ->  Seq Scan on rp1
+         Filter: ((a <> 1) AND (a <> 2))
+   ->  Seq Scan on rp2
+         Filter: ((a <> 1) AND (a <> 2))
+(7 rows)
+
+-- null partition should be eliminated due to strict <> clause.
+explain (costs off) select * from lp where a <> 'a';
+             QUERY PLAN             
+------------------------------------
+ Append
+   ->  Seq Scan on lp_ad
+         Filter: (a <> 'a'::bpchar)
+   ->  Seq Scan on lp_bc
+         Filter: (a <> 'a'::bpchar)
+   ->  Seq Scan on lp_ef
+         Filter: (a <> 'a'::bpchar)
+   ->  Seq Scan on lp_g
+         Filter: (a <> 'a'::bpchar)
+   ->  Seq Scan on lp_default
+         Filter: (a <> 'a'::bpchar)
+(11 rows)
+
+-- ensure we detect contradictions in clauses; a can't be NULL and NOT NULL.
+explain (costs off) select * from lp where a <> 'a' and a is null;
+        QUERY PLAN        
+--------------------------
+ Result
+   One-Time Filter: false
+(2 rows)
+
+explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Append
+   ->  Seq Scan on lp_bc
+         Filter: (((a <> 'a'::bpchar) AND (a <> 'd'::bpchar)) OR (a IS NULL))
+   ->  Seq Scan on lp_ef
+         Filter: (((a <> 'a'::bpchar) AND (a <> 'd'::bpchar)) OR (a IS NULL))
+   ->  Seq Scan on lp_g
+         Filter: (((a <> 'a'::bpchar) AND (a <> 'd'::bpchar)) OR (a IS NULL))
+   ->  Seq Scan on lp_null
+         Filter: (((a <> 'a'::bpchar) AND (a <> 'd'::bpchar)) OR (a IS NULL))
+   ->  Seq Scan on lp_default
+         Filter: (((a <> 'a'::bpchar) AND (a <> 'd'::bpchar)) OR (a IS NULL))
+(11 rows)
+
+-- check that it also works for a partitioned table that's not root,
+-- which in this case are partitions of rlp that are themselves
+-- list-partitioned on b
+explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
+                                                                QUERY PLAN                                                                
+------------------------------------------------------------------------------------------------------------------------------------------
+ Append
+   ->  Seq Scan on rlp3efgh
+         Filter: ((b IS NOT NULL) AND ((b)::text <> 'ab'::text) AND ((b)::text <> 'cd'::text) AND ((b)::text <> 'xy'::text) AND (a = 15))
+   ->  Seq Scan on rlp3_default
+         Filter: ((b IS NOT NULL) AND ((b)::text <> 'ab'::text) AND ((b)::text <> 'cd'::text) AND ((b)::text <> 'xy'::text) AND (a = 15))
+(5 rows)
+
+--
+-- different collations for different keys with same expression
+--
+create table coll_pruning_multi (a text) partition by range (substr(a, 1) collate "POSIX", substr(a, 1) collate "C");
+create table coll_pruning_multi1 partition of coll_pruning_multi for values from ('a', 'a') to ('a', 'e');
+create table coll_pruning_multi2 partition of coll_pruning_multi for values from ('a', 'e') to ('a', 'z');
+create table coll_pruning_multi3 partition of coll_pruning_multi for values from ('b', 'a') to ('b', 'e');
+-- no pruning, because no value for the leading key
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'e' collate "C";
+                       QUERY PLAN                       
+--------------------------------------------------------
+ Append
+   ->  Seq Scan on coll_pruning_multi1
+         Filter: (substr(a, 1) = 'e'::text COLLATE "C")
+   ->  Seq Scan on coll_pruning_multi2
+         Filter: (substr(a, 1) = 'e'::text COLLATE "C")
+   ->  Seq Scan on coll_pruning_multi3
+         Filter: (substr(a, 1) = 'e'::text COLLATE "C")
+(7 rows)
+
+-- pruning, with a value provided for the leading key
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'a' collate "POSIX";
+                         QUERY PLAN                         
+------------------------------------------------------------
+ Append
+   ->  Seq Scan on coll_pruning_multi1
+         Filter: (substr(a, 1) = 'a'::text COLLATE "POSIX")
+   ->  Seq Scan on coll_pruning_multi2
+         Filter: (substr(a, 1) = 'a'::text COLLATE "POSIX")
+(5 rows)
+
+-- pruning, with values provided for both keys
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'e' collate "C" and substr(a, 1) = 'a' collate "POSIX";
+                                               QUERY PLAN                                                
+---------------------------------------------------------------------------------------------------------
+ Append
+   ->  Seq Scan on coll_pruning_multi2
+         Filter: ((substr(a, 1) = 'e'::text COLLATE "C") AND (substr(a, 1) = 'a'::text COLLATE "POSIX"))
+(3 rows)
+
+--
+-- LIKE operators don't prune
+--
+create table like_op_noprune (a text) partition by list (a);
+create table like_op_noprune1 partition of like_op_noprune for values in ('ABC');
+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';
+             QUERY PLAN             
+------------------------------------
+ Append
+   ->  Seq Scan on like_op_noprune1
+         Filter: (a ~~ '%BC'::text)
+   ->  Seq Scan on like_op_noprune2
+         Filter: (a ~~ '%BC'::text)
+(5 rows)
+
+--
+-- tests wherein clause value requires a cross-type comparison function
+--
+create table lparted_by_int2 (a smallint) partition by list (a);
+create table lparted_by_int2_1 partition of lparted_by_int2 for values in (1);
+create table lparted_by_int2_16384 partition of lparted_by_int2 for values in (16384);
+explain (costs off) select * from lparted_by_int2 where a = 100000000000000;
+        QUERY PLAN        
+--------------------------
+ Result
+   One-Time Filter: false
+(2 rows)
+
+create table rparted_by_int2 (a smallint) partition by range (a);
+create table rparted_by_int2_1 partition of rparted_by_int2 for values from (1) to (10);
+create table rparted_by_int2_16384 partition of rparted_by_int2 for values from (10) to (16384);
+-- all partitions pruned
+explain (costs off) select * from rparted_by_int2 where a > 100000000000000;
+        QUERY PLAN        
+--------------------------
+ Result
+   One-Time Filter: false
+(2 rows)
+
+create table rparted_by_int2_maxvalue partition of rparted_by_int2 for values from (16384) to (maxvalue);
+-- all partitions but rparted_by_int2_maxvalue pruned
+explain (costs off) select * from rparted_by_int2 where a > 100000000000000;
+                   QUERY PLAN                    
+-------------------------------------------------
+ Append
+   ->  Seq Scan on rparted_by_int2_maxvalue
+         Filter: (a > '100000000000000'::bigint)
+(3 rows)
+
+drop table lp, coll_pruning, rlp, mc3p, mc2p, boolpart, rp, coll_pruning_multi, like_op_noprune, lparted_by_int2, rparted_by_int2;
+-- hash partitioning
+create table hp (a int, b text) partition by hash (a, b);
+create table hp0 partition of hp for values with (modulus 4, remainder 0);
+create table hp3 partition of hp for values with (modulus 4, remainder 3);
+create table hp1 partition of hp for values with (modulus 4, remainder 1);
+create table hp2 partition of hp for values with (modulus 4, remainder 2);
+insert into hp values (null, null);
+insert into hp values (1, null);
+insert into hp values (1, 'xxx');
+insert into hp values (null, 'xxx');
+insert into hp values (10, 'xxx');
+insert into hp values (10, 'yyy');
+select tableoid::regclass, * from hp order by 1;
+ tableoid | a  |  b  
+----------+----+-----
+ hp0      |    | 
+ hp0      |  1 | 
+ hp0      |  1 | xxx
+ hp3      | 10 | yyy
+ hp1      |    | xxx
+ hp2      | 10 | xxx
+(6 rows)
+
+-- partial keys won't prune, nor would non-equality conditions
+explain (costs off) select * from hp where a = 1;
+       QUERY PLAN        
+-------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: (a = 1)
+   ->  Seq Scan on hp1
+         Filter: (a = 1)
+   ->  Seq Scan on hp2
+         Filter: (a = 1)
+   ->  Seq Scan on hp3
+         Filter: (a = 1)
+(9 rows)
+
+explain (costs off) select * from hp where b = 'xxx';
+            QUERY PLAN             
+-----------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: (b = 'xxx'::text)
+   ->  Seq Scan on hp1
+         Filter: (b = 'xxx'::text)
+   ->  Seq Scan on hp2
+         Filter: (b = 'xxx'::text)
+   ->  Seq Scan on hp3
+         Filter: (b = 'xxx'::text)
+(9 rows)
+
+explain (costs off) select * from hp where a is null;
+         QUERY PLAN          
+-----------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: (a IS NULL)
+   ->  Seq Scan on hp1
+         Filter: (a IS NULL)
+   ->  Seq Scan on hp2
+         Filter: (a IS NULL)
+   ->  Seq Scan on hp3
+         Filter: (a IS NULL)
+(9 rows)
+
+explain (costs off) select * from hp where b is null;
+         QUERY PLAN          
+-----------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: (b IS NULL)
+   ->  Seq Scan on hp1
+         Filter: (b IS NULL)
+   ->  Seq Scan on hp2
+         Filter: (b IS NULL)
+   ->  Seq Scan on hp3
+         Filter: (b IS NULL)
+(9 rows)
+
+explain (costs off) select * from hp where a < 1 and b = 'xxx';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((a < 1) AND (b = 'xxx'::text))
+   ->  Seq Scan on hp1
+         Filter: ((a < 1) AND (b = 'xxx'::text))
+   ->  Seq Scan on hp2
+         Filter: ((a < 1) AND (b = 'xxx'::text))
+   ->  Seq Scan on hp3
+         Filter: ((a < 1) AND (b = 'xxx'::text))
+(9 rows)
+
+explain (costs off) select * from hp where a <> 1 and b = 'yyy';
+                    QUERY PLAN                    
+--------------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((a <> 1) AND (b = 'yyy'::text))
+   ->  Seq Scan on hp1
+         Filter: ((a <> 1) AND (b = 'yyy'::text))
+   ->  Seq Scan on hp2
+         Filter: ((a <> 1) AND (b = 'yyy'::text))
+   ->  Seq Scan on hp3
+         Filter: ((a <> 1) AND (b = 'yyy'::text))
+(9 rows)
+
+-- pruning should work if non-null values are provided for all the keys
+explain (costs off) select * from hp where a is null and b is null;
+                  QUERY PLAN                   
+-----------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((a IS NULL) AND (b IS NULL))
+(3 rows)
+
+explain (costs off) select * from hp where a = 1 and b is null;
+                QUERY PLAN                 
+-------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((b IS NULL) AND (a = 1))
+(3 rows)
+
+explain (costs off) select * from hp where a = 1 and b = 'xxx';
+                   QUERY PLAN                    
+-------------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((a = 1) AND (b = 'xxx'::text))
+(3 rows)
+
+explain (costs off) select * from hp where a is null and b = 'xxx';
+                     QUERY PLAN                      
+-----------------------------------------------------
+ Append
+   ->  Seq Scan on hp1
+         Filter: ((a IS NULL) AND (b = 'xxx'::text))
+(3 rows)
+
+explain (costs off) select * from hp where a = 10 and b = 'xxx';
+                    QUERY PLAN                    
+--------------------------------------------------
+ Append
+   ->  Seq Scan on hp2
+         Filter: ((a = 10) AND (b = 'xxx'::text))
+(3 rows)
+
+explain (costs off) select * from hp where a = 10 and b = 'yyy';
+                    QUERY PLAN                    
+--------------------------------------------------
+ Append
+   ->  Seq Scan on hp3
+         Filter: ((a = 10) AND (b = 'yyy'::text))
+(3 rows)
+
+explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and b = 'xxx') or (a is null and b is null);
+                                                       QUERY PLAN                                                        
+-------------------------------------------------------------------------------------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: (((a = 10) AND (b = 'yyy'::text)) OR ((a = 10) AND (b = 'xxx'::text)) OR ((a IS NULL) AND (b IS NULL)))
+   ->  Seq Scan on hp2
+         Filter: (((a = 10) AND (b = 'yyy'::text)) OR ((a = 10) AND (b = 'xxx'::text)) OR ((a IS NULL) AND (b IS NULL)))
+   ->  Seq Scan on hp3
+         Filter: (((a = 10) AND (b = 'yyy'::text)) OR ((a = 10) AND (b = 'xxx'::text)) OR ((a IS NULL) AND (b IS NULL)))
+(7 rows)
+
+-- hash partitiong pruning doesn't occur with <> operator clauses
+explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
+                    QUERY PLAN                     
+---------------------------------------------------
+ Append
+   ->  Seq Scan on hp0
+         Filter: ((a <> 1) AND (b <> 'xxx'::text))
+   ->  Seq Scan on hp1
+         Filter: ((a <> 1) AND (b <> 'xxx'::text))
+   ->  Seq Scan on hp2
+         Filter: ((a <> 1) AND (b <> 'xxx'::text))
+   ->  Seq Scan on hp3
+         Filter: ((a <> 1) AND (b <> 'xxx'::text))
+(9 rows)
+
+drop table hp;
index d4ef192fcdd14f57bd5e095380440803a5cb956f..ad5177715cdee93c524b873dfdb71a5a0ef7f7e7 100644 (file)
@@ -60,7 +60,7 @@ explain (costs off) select * from rlp where 1 > a;    /* commuted */
 explain (costs off) select * from rlp where a <= 1;
 explain (costs off) select * from rlp where a = 1;
 explain (costs off) select * from rlp where a = 1::bigint;     /* same as above */
-explain (costs off) select * from rlp where a = 1::numeric;    /* only null can be pruned */
+explain (costs off) select * from rlp where a = 1::numeric;        /* no pruning */
 explain (costs off) select * from rlp where a <= 10;
 explain (costs off) select * from rlp where a > 10;
 explain (costs off) select * from rlp where a < 15;
@@ -152,4 +152,125 @@ explain (costs off) select * from boolpart where a is not true and a is not fals
 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;
+--
+-- some more cases
+--
+
+--
+-- pruning for partitioned table appearing inside a sub-query
+--
+-- pruning won't work for mc3p, because some keys are Params
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.a = t1.b and abs(t2.b) = 1 and t2.c = 1) s where t1.a = 1;
+
+-- pruning should work fine, because values for a prefix of keys (a, b) are
+-- available
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.c = t1.b and abs(t2.b) = 1 and t2.a = 1) s where t1.a = 1;
+
+-- also here, because values for all keys are provided
+explain (costs off) select * from mc2p t1, lateral (select count(*) from mc3p t2 where t2.a = 1 and abs(t2.b) = 1 and t2.c = 1) s where t1.a = 1;
+
+--
+-- pruning with clauses containing <> operator
+--
+
+-- doesn't prune range partitions
+create table rp (a int) partition by range (a);
+create table rp0 partition of rp for values from (minvalue) to (1);
+create table rp1 partition of rp for values from (1) to (2);
+create table rp2 partition of rp for values from (2) to (maxvalue);
+
+explain (costs off) select * from rp where a <> 1;
+explain (costs off) select * from rp where a <> 1 and a <> 2;
+
+-- null partition should be eliminated due to strict <> clause.
+explain (costs off) select * from lp where a <> 'a';
+
+-- ensure we detect contradictions in clauses; a can't be NULL and NOT NULL.
+explain (costs off) select * from lp where a <> 'a' and a is null;
+explain (costs off) select * from lp where (a <> 'a' and a <> 'd') or a is null;
+
+-- check that it also works for a partitioned table that's not root,
+-- which in this case are partitions of rlp that are themselves
+-- list-partitioned on b
+explain (costs off) select * from rlp where a = 15 and b <> 'ab' and b <> 'cd' and b <> 'xy' and b is not null;
+
+--
+-- different collations for different keys with same expression
+--
+create table coll_pruning_multi (a text) partition by range (substr(a, 1) collate "POSIX", substr(a, 1) collate "C");
+create table coll_pruning_multi1 partition of coll_pruning_multi for values from ('a', 'a') to ('a', 'e');
+create table coll_pruning_multi2 partition of coll_pruning_multi for values from ('a', 'e') to ('a', 'z');
+create table coll_pruning_multi3 partition of coll_pruning_multi for values from ('b', 'a') to ('b', 'e');
+
+-- no pruning, because no value for the leading key
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'e' collate "C";
+
+-- pruning, with a value provided for the leading key
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'a' collate "POSIX";
+
+-- pruning, with values provided for both keys
+explain (costs off) select * from coll_pruning_multi where substr(a, 1) = 'e' collate "C" and substr(a, 1) = 'a' collate "POSIX";
+
+--
+-- LIKE operators don't prune
+--
+create table like_op_noprune (a text) partition by list (a);
+create table like_op_noprune1 partition of like_op_noprune for values in ('ABC');
+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';
+
+--
+-- tests wherein clause value requires a cross-type comparison function
+--
+create table lparted_by_int2 (a smallint) partition by list (a);
+create table lparted_by_int2_1 partition of lparted_by_int2 for values in (1);
+create table lparted_by_int2_16384 partition of lparted_by_int2 for values in (16384);
+explain (costs off) select * from lparted_by_int2 where a = 100000000000000;
+
+create table rparted_by_int2 (a smallint) partition by range (a);
+create table rparted_by_int2_1 partition of rparted_by_int2 for values from (1) to (10);
+create table rparted_by_int2_16384 partition of rparted_by_int2 for values from (10) to (16384);
+-- all partitions pruned
+explain (costs off) select * from rparted_by_int2 where a > 100000000000000;
+create table rparted_by_int2_maxvalue partition of rparted_by_int2 for values from (16384) to (maxvalue);
+-- all partitions but rparted_by_int2_maxvalue pruned
+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;
+
+-- hash partitioning
+create table hp (a int, b text) partition by hash (a, b);
+create table hp0 partition of hp for values with (modulus 4, remainder 0);
+create table hp3 partition of hp for values with (modulus 4, remainder 3);
+create table hp1 partition of hp for values with (modulus 4, remainder 1);
+create table hp2 partition of hp for values with (modulus 4, remainder 2);
+
+insert into hp values (null, null);
+insert into hp values (1, null);
+insert into hp values (1, 'xxx');
+insert into hp values (null, 'xxx');
+insert into hp values (10, 'xxx');
+insert into hp values (10, 'yyy');
+select tableoid::regclass, * from hp order by 1;
+
+-- partial keys won't prune, nor would non-equality conditions
+explain (costs off) select * from hp where a = 1;
+explain (costs off) select * from hp where b = 'xxx';
+explain (costs off) select * from hp where a is null;
+explain (costs off) select * from hp where b is null;
+explain (costs off) select * from hp where a < 1 and b = 'xxx';
+explain (costs off) select * from hp where a <> 1 and b = 'yyy';
+
+-- pruning should work if non-null values are provided for all the keys
+explain (costs off) select * from hp where a is null and b is null;
+explain (costs off) select * from hp where a = 1 and b is null;
+explain (costs off) select * from hp where a = 1 and b = 'xxx';
+explain (costs off) select * from hp where a is null and b = 'xxx';
+explain (costs off) select * from hp where a = 10 and b = 'xxx';
+explain (costs off) select * from hp where a = 10 and b = 'yyy';
+explain (costs off) select * from hp where (a = 10 and b = 'yyy') or (a = 10 and b = 'xxx') or (a is null and b is null);
+
+-- hash partitiong pruning doesn't occur with <> operator clauses
+explain (costs off) select * from hp where a <> 1 and b <> 'xxx';
+
+drop table hp;