diff --git a/src/backend/Makefile b/src/backend/Makefile index a4b6d1658c..42a0748ade 100644 --- a/src/backend/Makefile +++ b/src/backend/Makefile @@ -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 diff --git a/src/backend/catalog/partition.c b/src/backend/catalog/partition.c index 39ee773d93..a60e5c20d6 100644 --- a/src/backend/catalog/partition.c +++ b/src/backend/catalog/partition.c @@ -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" @@ -55,89 +56,6 @@ #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) { diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index d2e4aa3c2f..9287baaedc 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -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 * @@ -2278,21 +2310,6 @@ _copyAppendRelInfo(const AppendRelInfo *from) } /* - * _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 */ static PlaceHolderInfo * @@ -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; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index f2dd9035df..d758515cfd 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -916,16 +916,6 @@ _equalAppendRelInfo(const AppendRelInfo *a, const AppendRelInfo *b) } 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) { COMPARE_SCALAR_FIELD(phid); @@ -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; diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c index f2f8227eb2..51c418778a 100644 --- a/src/backend/nodes/nodeFuncs.c +++ b/src/backend/nodes/nodeFuncs.c @@ -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; diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index a6a1c16164..03a91c3352 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -1711,6 +1711,28 @@ _outFromExpr(StringInfo str, const FromExpr *node) } 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) { WRITE_NODE_TYPE("ONCONFLICTEXPR"); @@ -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 @@ -2560,16 +2583,6 @@ _outAppendRelInfo(StringInfo str, const AppendRelInfo *node) } 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) { WRITE_NODE_TYPE("PLACEHOLDERINFO"); @@ -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; diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c index 37e3568595..2812dc9646 100644 --- a/src/backend/nodes/readfuncs.c +++ b/src/backend/nodes/readfuncs.c @@ -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)) diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c index c4e4db15a6..65a34a255d 100644 --- a/src/backend/optimizer/path/allpaths.c +++ b/src/backend/optimizer/path/allpaths.c @@ -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,6 +875,8 @@ 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(); @@ -881,6 +884,31 @@ set_append_rel_size(PlannerInfo *root, RelOptInfo *rel, 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. * * We handle width estimates by weighting the widths of different child @@ -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)) + /* + * 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) { - /* - * 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) + 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)); } diff --git a/src/backend/optimizer/path/indxpath.c b/src/backend/optimizer/path/indxpath.c index 594ac8eacb..ec3f60d311 100644 --- a/src/backend/optimizer/path/indxpath.c +++ b/src/backend/optimizer/path/indxpath.c @@ -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)) diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c index 15c8d34c70..008492bad5 100644 --- a/src/backend/optimizer/plan/planner.c +++ b/src/backend/optimizer/plan/planner.c @@ -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); } /* @@ -1503,6 +1505,15 @@ inheritance_planner(PlannerInfo *root) 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 * just its modified subquery RTEs to final_rtable. @@ -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, @@ -6145,65 +6171,6 @@ done: } /* - * 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 * * Add non-partial paths to grouping relation. diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c index 5236ab378e..67e47887fc 100644 --- a/src/backend/optimizer/prep/prepunion.c +++ b/src/backend/optimizer/prep/prepunion.c @@ -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); diff --git a/src/backend/optimizer/util/plancat.c b/src/backend/optimizer/util/plancat.c index 8a6baa7bea..52e4cca49a 100644 --- a/src/backend/optimizer/util/plancat.c +++ b/src/backend/optimizer/util/plancat.c @@ -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); diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c index da8f0f93fc..b9aa7486ba 100644 --- a/src/backend/optimizer/util/relnode.c +++ b/src/backend/optimizer/util/relnode.c @@ -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 index 0000000000..429207c4eb --- /dev/null +++ b/src/backend/partitioning/Makefile @@ -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 index 0000000000..cfde207df3 --- /dev/null +++ b/src/backend/partitioning/partprune.c @@ -0,0 +1,2780 @@ +/*------------------------------------------------------------------------- + * + * 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/partition.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 "parser/parse_coerce.h" +#include "parser/parsetree.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(RelOptInfo *rel, + GeneratePruningStepsContext *context, + List *clauses, + bool *constfalse); +static PartitionPruneStep *gen_prune_step_op(GeneratePruningStepsContext *context, + int 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, + int 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, + int 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, + int opstrategy, Datum *values, int nvalues, + FmgrInfo *partsupfunc, Bitmapset *nullkeys); +static PruneStepResult *get_matching_list_bounds(PartitionPruneContext *context, + int opstrategy, Datum value, int nvalues, + FmgrInfo *partsupfunc, Bitmapset *nullkeys); +static PruneStepResult *get_matching_range_bounds(PartitionPruneContext *context, + int 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 any of the clauses in the input list is a pseudo-constant "false", + * *constfalse is set to true upon return. + */ +List * +gen_partprune_steps(RelOptInfo *rel, List *clauses, bool *constfalse) +{ + 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. */ + *constfalse = false; + gen_partprune_steps_internal(rel, &context, clauses, constfalse); + + 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 constfalse; + PartitionPruneContext context; + Bitmapset *partindexes; + int i; + + Assert(clauses != NIL); + Assert(rel->part_scheme != NULL); + + /* Quick exit. */ + if (rel->nparts == 0) + return NULL; + + /* process clauses */ + pruning_steps = gen_partprune_steps(rel, clauses, &constfalse); + if (constfalse) + 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 unique step identifier, across recursive calls. + * + * If we find clauses that are mutually contradictory, or a pseudoconstant + * clause that contains false, we set *constfalse to true and return NIL (no + * pruning steps). Caller should consider all partitions as pruned in that + * case. + * + * 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(RelOptInfo *rel, GeneratePruningStepsContext *context, + List *clauses, bool *constfalse) +{ + 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; + + 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)) + { + *constfalse = 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_constfalse = true; + ListCell *lc1; + + /* + * Get pruning step for each arg. If we get constfalse for + * all args, it means the OR expression is false as a whole. + */ + foreach(lc1, ((BoolExpr *) clause)->args) + { + Expr *arg = lfirst(lc1); + bool arg_constfalse; + List *argsteps; + + arg_constfalse = false; + argsteps = + gen_partprune_steps_internal(rel, context, + list_make1(arg), + &arg_constfalse); + if (!arg_constfalse) + all_args_constfalse = 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_constfalse 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_constfalse) + 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); + } + } + + *constfalse = all_args_constfalse; + + /* Check if any contradicting clauses were found */ + if (*constfalse) + 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. + */ + *constfalse = false; + argsteps = gen_partprune_steps_internal(rel, context, args, + constfalse); + if (*constfalse) + return NIL; + + foreach(lc1, argsteps) + { + PartitionPruneStep *step = lfirst(lc1); + + arg_stepids = lappend_int(arg_stepids, step->step_id); + } + + if (arg_stepids) + { + 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)) + { + *constfalse = 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)) + { + *constfalse = true; + return NIL; + } + nullkeys = bms_add_member(nullkeys, i); + } + else + { + /* check for conflicting IS NULL */ + if (bms_is_member(i, nullkeys)) + { + *constfalse = 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. */ + *constfalse = 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, 0, 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, 0, 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; +} + +/* + * 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, + int 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; +} + +/* + * 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 + * + * '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], + *hash_clauses[HTMaxStrategyNumber]; + 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 - 1] != NIL) + last = llast(btree_clauses[pc->op_strategy - 1]); + + if (last == NULL || + i == last->keyno || i == last->keyno + 1) + btree_clauses[pc->op_strategy - 1] = + lappend(btree_clauses[pc->op_strategy - 1], 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 - 1] = + lappend(hash_clauses[pc->op_strategy - 1], 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 - 1]; + List *le_clauses = btree_clauses[BTLessEqualStrategyNumber - 1]; + List *ge_clauses = btree_clauses[BTGreaterEqualStrategyNumber - 1]; + + /* + * 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 (i = 0; i < BTMaxStrategyNumber; i++) + { + PartClauseInfo *pc; + List *pc_steps; + + foreach(lc, btree_clauses[i]) + { + ListCell *lc1; + List *prefix = NIL; + + /* Clause under consideration. */ + pc = lfirst(lc); + + /* + * 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 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 (i == BTGreaterStrategyNumber - 1 || + i == BTGreaterEqualStrategyNumber - 1) + { + 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 == i + 1); + pc_steps = get_steps_using_prefix(context, i + 1, + 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 - 1]; + + /* 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; + } + + /* Finally, add a combine step to mutualy AND opsteps, 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 constfalse = false; + + orexpr = makeBoolExpr(OR_EXPR, elem_clauses, -1); + *clause_steps = + gen_partprune_steps_internal(rel, context, list_make1(orexpr), + &constfalse); + if (constfalse) + return PARTCLAUSE_MATCH_CONTRADICT; + + Assert(list_length(*clause_steps) == 1); + return PARTCLAUSE_MATCH_STEPS; + } + else + { + bool constfalse = false; + + *clause_steps = + gen_partprune_steps_internal(rel, context, elem_clauses, + &constfalse); + if (constfalse) + 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, + int 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, + int 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) + { + 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; + } + + result = + list_concat(result, + 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)); + } + } + 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, + int 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, + int 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, + int 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; +} diff --git a/src/include/catalog/partition.h b/src/include/catalog/partition.h index cd15faa7a1..b25e25bf9d 100644 --- a/src/include/catalog/partition.h +++ b/src/include/catalog/partition.h @@ -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); diff --git a/src/include/catalog/pg_opfamily.h b/src/include/catalog/pg_opfamily.h index b544474254..5b20dd77a1 100644 --- a/src/include/catalog/pg_opfamily.h +++ b/src/include/catalog/pg_opfamily.h @@ -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 * ---------------- diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h index b1e3d53f78..4fc2de7184 100644 --- a/src/include/nodes/nodes.h +++ b/src/include/nodes/nodes.h @@ -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, diff --git a/src/include/nodes/primnodes.h b/src/include/nodes/primnodes.h index 1b4b0d75af..7c4540b261 100644 --- a/src/include/nodes/primnodes.h +++ b/src/include/nodes/primnodes.h @@ -1506,4 +1506,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; + + int 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 */ diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h index a2dde70de5..acb8814924 100644 --- a/src/include/nodes/relation.h +++ b/src/include/nodes/relation.h @@ -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 */ @@ -2122,27 +2133,6 @@ typedef struct AppendRelInfo } 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. * This stores info that is needed centrally rather than in each copy of the diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h index 07a3bc0627..c090396e13 100644 --- a/src/include/optimizer/planner.h +++ b/src/include/optimizer/planner.h @@ -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 index 0000000000..c76014d4a8 --- /dev/null +++ b/src/include/partitioning/partbounds.h @@ -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 index 0000000000..b272ae9a90 --- /dev/null +++ b/src/include/partitioning/partprune.h @@ -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 *constfalse); + +#endif /* PARTPRUNE_H */ diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out index 5e57b9a465..b2b912ed5c 100644 --- a/src/test/regress/expected/inherit.out +++ b/src/test/regress/expected/inherit.out @@ -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 diff --git a/src/test/regress/expected/partition_prune.out b/src/test/regress/expected/partition_prune.out index 09517775b6..2d77b3edd4 100644 --- a/src/test/regress/expected/partition_prune.out +++ b/src/test/regress/expected/partition_prune.out @@ -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; diff --git a/src/test/regress/sql/partition_prune.sql b/src/test/regress/sql/partition_prune.sql index d4ef192fcd..ad5177715c 100644 --- a/src/test/regress/sql/partition_prune.sql +++ b/src/test/regress/sql/partition_prune.sql @@ -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;