From 24a86111f31fc3e95c02dc3c75f1be390ebbd09b Mon Sep 17 00:00:00 2001 From: Tomas Vondra Date: Wed, 25 Mar 2020 22:43:28 +0100 Subject: [PATCH 1/3] v33 --- src/backend/nodes/outfuncs.c | 2 + src/backend/optimizer/path/joinrels.c | 242 +- src/backend/optimizer/util/inherit.c | 2 + src/backend/optimizer/util/relnode.c | 43 +- src/backend/partitioning/partbounds.c | 1838 ++++++++++++ src/include/nodes/pathnodes.h | 8 +- src/include/partitioning/partbounds.h | 9 + src/test/regress/expected/partition_join.out | 2608 +++++++++++++++++- src/test/regress/sql/partition_join.sql | 639 ++++- 9 files changed, 5315 insertions(+), 76 deletions(-) diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index e084c3f069..79c768409c 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -2288,6 +2288,8 @@ _outRelOptInfo(StringInfo str, const RelOptInfo *node) WRITE_BOOL_FIELD(has_eclass_joins); WRITE_BOOL_FIELD(consider_partitionwise_join); WRITE_BITMAPSET_FIELD(top_parent_relids); + WRITE_BOOL_FIELD(merged); + WRITE_BITMAPSET_FIELD(all_partrels); WRITE_NODE_FIELD(partitioned_child_rels); } diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c index a21c295b99..530ebed245 100644 --- a/src/backend/optimizer/path/joinrels.c +++ b/src/backend/optimizer/path/joinrels.c @@ -48,6 +48,9 @@ static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root, Relids left_relids, Relids right_relids); static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op); +static void get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel, + RelOptInfo *rel1, RelOptInfo *rel2, + List **parts1, List **parts2); /* @@ -1357,25 +1360,30 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2, { bool rel1_is_simple = IS_SIMPLE_REL(rel1); bool rel2_is_simple = IS_SIMPLE_REL(rel2); - int nparts; + bool merged = false; + List *parts1 = NIL; + List *parts2 = NIL; + ListCell *lcr1 = NULL; + ListCell *lcr2 = NULL; int cnt_parts; /* Guard against stack overflow due to overly deep partition hierarchy. */ check_stack_depth(); /* Nothing to do, if the join relation is not partitioned. */ - if (!IS_PARTITIONED_REL(joinrel)) + if (joinrel->part_scheme == NULL || joinrel->nparts == 0) return; /* The join relation should have consider_partitionwise_join set. */ Assert(joinrel->consider_partitionwise_join); /* - * Since this join relation is partitioned, all the base relations - * participating in this join must be partitioned and so are all the - * intermediate join relations. + * We can not perform partitionwise join if either of the joining relations + * is not partitioned. */ - Assert(IS_PARTITIONED_REL(rel1) && IS_PARTITIONED_REL(rel2)); + if (!IS_PARTITIONED_REL(rel1) || !IS_PARTITIONED_REL(rel2)) + return; + Assert(REL_HAS_ALL_PART_PROPS(rel1) && REL_HAS_ALL_PART_PROPS(rel2)); /* The joining relations should have consider_partitionwise_join set. */ @@ -1390,34 +1398,104 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2, joinrel->part_scheme == rel2->part_scheme); /* - * Since we allow partitionwise join only when the partition bounds of the - * joining relations exactly match, the partition bounds of the join - * should match those of the joining relations. + * If we don't have the partition bounds for the join rel yet, try to + * compute those along with pairs of partitions to be joined. */ - Assert(partition_bounds_equal(joinrel->part_scheme->partnatts, - joinrel->part_scheme->parttyplen, - joinrel->part_scheme->parttypbyval, - joinrel->boundinfo, rel1->boundinfo)); - Assert(partition_bounds_equal(joinrel->part_scheme->partnatts, - joinrel->part_scheme->parttyplen, - joinrel->part_scheme->parttypbyval, - joinrel->boundinfo, rel2->boundinfo)); + if (joinrel->nparts == -1) + { + PartitionScheme part_scheme = joinrel->part_scheme; + PartitionBoundInfo boundinfo = NULL; + int nparts = 0; + + Assert(joinrel->boundinfo == NULL); + Assert(joinrel->part_rels == NULL); + + /* + * See if the partition bounds for inputs are exactly the same, in + * which case we don't need to work hard: the join rel have the same + * partition bounds as inputs, and the partitions with the same + * cardinal positions form the pairs. + * + * Note: even in cases where one or both inputs have merged bounds, + * it would be possible for both the bounds to be exactly the same, but + * it seems unlikely to be worth the cycles to check. + */ + if (!rel1->merged && + !rel2->merged && + rel1->nparts == rel2->nparts && + partition_bounds_equal(part_scheme->partnatts, + part_scheme->parttyplen, + part_scheme->parttypbyval, + rel1->boundinfo, rel2->boundinfo)) + { + boundinfo = rel1->boundinfo; + nparts = rel1->nparts; + } + else + { + /* Try merging the partition bounds for inputs. */ + boundinfo = partition_bounds_merge(part_scheme->partnatts, + part_scheme->partsupfunc, + part_scheme->partcollation, + rel1, rel2, + parent_sjinfo->jointype, + &parts1, &parts2); + if (boundinfo == NULL) + { + joinrel->nparts = 0; + return; + } + nparts = list_length(parts1); + merged = true; + } + + Assert(nparts > 0); + joinrel->boundinfo = boundinfo; + joinrel->merged = merged; + joinrel->nparts = nparts; + joinrel->part_rels = + (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * nparts); + } + else + { + Assert(joinrel->nparts > 0); + Assert(joinrel->boundinfo); + Assert(joinrel->part_rels); + + /* + * If the join rel's merged flag is true, it means inputs are not + * guaranteed to have the same partition bounds, therefore we can't + * assume that the partitions at the same cardinal positions form the + * pairs; let get_matching_part_pairs() generate the pairs. Otherwise, + * nothing to do since we can assume that. + */ + if (joinrel->merged) + { + get_matching_part_pairs(root, joinrel, rel1, rel2, + &parts1, &parts2); + Assert(list_length(parts1) == joinrel->nparts); + Assert(list_length(parts2) == joinrel->nparts); + merged = true; + } + } - nparts = joinrel->nparts; + if (merged) + { + lcr1 = list_head(parts1); + lcr2 = list_head(parts2); + } /* * Create child-join relations for this partitioned join, if those don't * exist. Add paths to child-joins for a pair of child relations * corresponding to the given pair of parent relations. */ - for (cnt_parts = 0; cnt_parts < nparts; cnt_parts++) + for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++) { - RelOptInfo *child_rel1 = rel1->part_rels[cnt_parts]; - RelOptInfo *child_rel2 = rel2->part_rels[cnt_parts]; - bool rel1_empty = (child_rel1 == NULL || - IS_DUMMY_REL(child_rel1)); - bool rel2_empty = (child_rel2 == NULL || - IS_DUMMY_REL(child_rel2)); + RelOptInfo *child_rel1; + RelOptInfo *child_rel2; + bool rel1_empty; + bool rel2_empty; SpecialJoinInfo *child_sjinfo; List *child_restrictlist; RelOptInfo *child_joinrel; @@ -1425,6 +1503,22 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2, AppendRelInfo **appinfos; int nappinfos; + if (merged) + { + child_rel1 = lfirst_node(RelOptInfo, lcr1); + child_rel2 = lfirst_node(RelOptInfo, lcr2); + lcr1 = lnext(parts1, lcr1); + lcr2 = lnext(parts2, lcr2); + } + else + { + child_rel1 = rel1->part_rels[cnt_parts]; + child_rel2 = rel2->part_rels[cnt_parts]; + } + + rel1_empty = (child_rel1 == NULL || IS_DUMMY_REL(child_rel1)); + rel2_empty = (child_rel2 == NULL || IS_DUMMY_REL(child_rel2)); + /* * Check for cases where we can prove that this segment of the join * returns no rows, due to one or both inputs being empty (including @@ -1522,6 +1616,8 @@ try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, RelOptInfo *rel2, child_sjinfo, child_sjinfo->jointype); joinrel->part_rels[cnt_parts] = child_joinrel; + joinrel->all_partrels = bms_add_members(joinrel->all_partrels, + child_joinrel->relids); } Assert(bms_equal(child_joinrel->relids, child_joinrelids)); @@ -1738,3 +1834,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op) return -1; } + +/* + * get_matching_part_pairs + * Generate pairs of partitions to be joined from the two inputs + */ +static void +get_matching_part_pairs(PlannerInfo *root, RelOptInfo *joinrel, + RelOptInfo *rel1, RelOptInfo *rel2, + List **parts1, List **parts2) +{ + bool rel1_is_simple = IS_SIMPLE_REL(rel1); + bool rel2_is_simple = IS_SIMPLE_REL(rel2); + int cnt_parts; + + *parts1 = NIL; + *parts2 = NIL; + + for (cnt_parts = 0; cnt_parts < joinrel->nparts; cnt_parts++) + { + RelOptInfo *child_joinrel = joinrel->part_rels[cnt_parts]; + RelOptInfo *child_rel1; + RelOptInfo *child_rel2; + Relids child_relids1; + Relids child_relids2; + + /* + * If this segment of the join is empty, it means that this segment + * was ignored when previously creating child-join paths for it in + * try_partitionwise_join() as it would not contribute to the join + * result, due to one or both inputs being empty; add NULL to each of + * the given lists so that this segment will be ignored again in that + * function. + */ + if (!child_joinrel) + { + *parts1 = lappend(*parts1, NULL); + *parts2 = lappend(*parts2, NULL); + continue; + } + + /* + * Get a relids set of partition(s) involved in this join segment that + * are from the rel1 side. + */ + child_relids1 = bms_intersect(child_joinrel->relids, + rel1->all_partrels); + Assert(bms_num_members(child_relids1) == bms_num_members(rel1->relids)); + + /* + * Get a child rel for rel1 with the relids. Note that we should have + * the child rel even if rel1 is a join rel, because in that case the + * partitions specified in the relids would have matching/overlapping + * boundaries, so the specified partitions should be considered as ones + * to be joined when planning partitionwise joins of rel1, meaning that + * the child rel would have been built by the time we get here. + */ + if (rel1_is_simple) + { + int varno = bms_singleton_member(child_relids1); + + child_rel1 = find_base_rel(root, varno); + } + else + child_rel1 = find_join_rel(root, child_relids1); + Assert(child_rel1); + + /* + * Get a relids set of partition(s) involved in this join segment that + * are from the rel2 side. + */ + child_relids2 = bms_intersect(child_joinrel->relids, + rel2->all_partrels); + Assert(bms_num_members(child_relids2) == bms_num_members(rel2->relids)); + + /* + * Get a child rel for rel2 with the relids. See above comments. + */ + if (rel2_is_simple) + { + int varno = bms_singleton_member(child_relids2); + + child_rel2 = find_base_rel(root, varno); + } + else + child_rel2 = find_join_rel(root, child_relids2); + Assert(child_rel2); + + /* + * The join of rel1 and rel2 is legal, so is the join of the child + * rels obtained above; add them to the given lists as a join pair + * producing this join segment. + */ + *parts1 = lappend(*parts1, child_rel1); + *parts2 = lappend(*parts2, child_rel2); + } +} diff --git a/src/backend/optimizer/util/inherit.c b/src/backend/optimizer/util/inherit.c index 7db67fdf34..3132fd35a5 100644 --- a/src/backend/optimizer/util/inherit.c +++ b/src/backend/optimizer/util/inherit.c @@ -376,6 +376,8 @@ expand_partitioned_rtentry(PlannerInfo *root, RelOptInfo *relinfo, /* Create the otherrel RelOptInfo too. */ childrelinfo = build_simple_rel(root, childRTindex, relinfo); relinfo->part_rels[i] = childrelinfo; + relinfo->all_partrels = bms_add_members(relinfo->all_partrels, + childrelinfo->relids); /* If this child is itself partitioned, recurse */ if (childrel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE) diff --git a/src/backend/optimizer/util/relnode.c b/src/backend/optimizer/util/relnode.c index 374f93890b..0e4944ac8e 100644 --- a/src/backend/optimizer/util/relnode.c +++ b/src/backend/optimizer/util/relnode.c @@ -240,10 +240,12 @@ build_simple_rel(PlannerInfo *root, int relid, RelOptInfo *parent) rel->has_eclass_joins = false; rel->consider_partitionwise_join = false; /* might get changed later */ rel->part_scheme = NULL; - rel->nparts = 0; + rel->nparts = -1; rel->boundinfo = NULL; + rel->merged = false; rel->partition_qual = NIL; rel->part_rels = NULL; + rel->all_partrels = NULL; rel->partexprs = NULL; rel->nullable_partexprs = NULL; rel->partitioned_child_rels = NIL; @@ -653,10 +655,12 @@ build_join_rel(PlannerInfo *root, joinrel->consider_partitionwise_join = false; /* might get changed later */ joinrel->top_parent_relids = NULL; joinrel->part_scheme = NULL; - joinrel->nparts = 0; + joinrel->nparts = -1; joinrel->boundinfo = NULL; + joinrel->merged = false; joinrel->partition_qual = NIL; joinrel->part_rels = NULL; + joinrel->all_partrels = NULL; joinrel->partexprs = NULL; joinrel->nullable_partexprs = NULL; joinrel->partitioned_child_rels = NIL; @@ -829,10 +833,12 @@ build_child_join_rel(PlannerInfo *root, RelOptInfo *outer_rel, joinrel->consider_partitionwise_join = false; /* might get changed later */ joinrel->top_parent_relids = NULL; joinrel->part_scheme = NULL; - joinrel->nparts = 0; + joinrel->nparts = -1; joinrel->boundinfo = NULL; + joinrel->merged = false; joinrel->partition_qual = NIL; joinrel->part_rels = NULL; + joinrel->all_partrels = NULL; joinrel->partexprs = NULL; joinrel->nullable_partexprs = NULL; joinrel->partitioned_child_rels = NIL; @@ -1639,7 +1645,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, * of the way the query planner deduces implied equalities and reorders * the joins. Please see optimizer/README for details. */ - if (!IS_PARTITIONED_REL(outer_rel) || !IS_PARTITIONED_REL(inner_rel) || + if (outer_rel->part_scheme == NULL || inner_rel->part_scheme == NULL || !outer_rel->consider_partitionwise_join || !inner_rel->consider_partitionwise_join || outer_rel->part_scheme != inner_rel->part_scheme || @@ -1652,24 +1658,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, part_scheme = outer_rel->part_scheme; - Assert(REL_HAS_ALL_PART_PROPS(outer_rel) && - REL_HAS_ALL_PART_PROPS(inner_rel)); - - /* - * For now, our partition matching algorithm can match partitions only - * when the partition bounds of the joining relations are exactly same. - * So, bail out otherwise. - */ - if (outer_rel->nparts != inner_rel->nparts || - !partition_bounds_equal(part_scheme->partnatts, - part_scheme->parttyplen, - part_scheme->parttypbyval, - outer_rel->boundinfo, inner_rel->boundinfo)) - { - Assert(!IS_PARTITIONED_REL(joinrel)); - return; - } - /* * This function will be called only once for each joinrel, hence it * should not have partition scheme, partition bounds, partition key @@ -1680,18 +1668,17 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, !joinrel->boundinfo); /* - * Join relation is partitioned using the same partitioning scheme as the - * joining relations and has same bounds. + * If the join relation is partitioned, it use the same partitioning scheme + * as the joining relations. + * + * Note: we calculate the partition bounds, number of partitions, and + * child-join relations of the join relation in try_partitionwise_join(). */ joinrel->part_scheme = part_scheme; - joinrel->boundinfo = outer_rel->boundinfo; partnatts = joinrel->part_scheme->partnatts; joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts); joinrel->nullable_partexprs = (List **) palloc0(sizeof(List *) * partnatts); - joinrel->nparts = outer_rel->nparts; - joinrel->part_rels = - (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts); /* * Set the consider_partitionwise_join flag. diff --git a/src/backend/partitioning/partbounds.c b/src/backend/partitioning/partbounds.c index 4c47f54a57..24dbc2c8f3 100644 --- a/src/backend/partitioning/partbounds.c +++ b/src/backend/partitioning/partbounds.c @@ -26,6 +26,7 @@ #include "miscadmin.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" +#include "nodes/pathnodes.h" #include "parser/parse_coerce.h" #include "partitioning/partbounds.h" #include "partitioning/partdesc.h" @@ -68,6 +69,25 @@ typedef struct PartitionRangeBound bool lower; /* this is the lower (vs upper) bound */ } PartitionRangeBound; +/* Per-partitioned-relation data for merge_list_bounds()/merge_range_bounds() */ +typedef struct PartitionMap +{ + int nparts; /* number of partitions */ + int *merged_indexes; /* indexes of merged partitions */ + bool *merged; /* flags to indicate whether partitions are + * merged with non-dummy partitions */ + bool did_remapping; /* did we re-map partitions? */ + int *old_indexes; /* old indexes of merged partitions if + * did_remapping */ +} PartitionMap; + +/* Macro for comparing two range bounds */ +#define compare_range_bounds(partnatts, partsupfunc, partcollations, \ + bound1, bound2) \ + (partition_rbound_cmp(partnatts, partsupfunc, partcollations, \ + (bound1)->datums, (bound1)->kind, (bound1)->lower, \ + bound2)) + static int32 qsort_partition_hbound_cmp(const void *a, const void *b); static int32 qsort_partition_list_value_cmp(const void *a, const void *b, void *arg); @@ -79,6 +99,116 @@ static PartitionBoundInfo create_list_bounds(PartitionBoundSpec **boundspecs, int nparts, PartitionKey key, int **mapping); static PartitionBoundInfo create_range_bounds(PartitionBoundSpec **boundspecs, int nparts, PartitionKey key, int **mapping); +static PartitionBoundInfo merge_list_bounds(FmgrInfo *partsupfunc, + Oid *collations, + RelOptInfo *outer_rel, + RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, + List **inner_parts); +static PartitionBoundInfo merge_range_bounds(int partnatts, + FmgrInfo *partsupfuncs, + Oid *partcollations, + RelOptInfo *outer_rel, + RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, + List **inner_parts); +static void init_partition_map(RelOptInfo *rel, PartitionMap *map); +static void free_partition_map(PartitionMap *map); +static bool is_dummy_partition(RelOptInfo *rel, int part_index); +static int merge_matching_partitions(PartitionMap *outer_map, + PartitionMap *inner_map, + int outer_part, + int inner_part, + int *next_index); +static int process_outer_partition(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int outer_index, + int inner_default, + JoinType jointype, + int *next_index, + int *default_index); +static int process_inner_partition(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int inner_index, + int outer_default, + JoinType jointype, + int *next_index, + int *default_index); +static void merge_null_partitions(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_null, + bool inner_has_null, + int outer_null, + int inner_null, + JoinType jointype, + int *next_index, + int *null_index); +static void merge_default_partitions(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int outer_default, + int inner_default, + JoinType jointype, + int *next_index, + int *default_index); +static int merge_partition_with_dummy(PartitionMap *map, int index, + int *next_index); +static void fix_merged_indexes(PartitionMap *outer_map, + PartitionMap *inner_map, + int nmerged, List *merged_indexes); +static void generate_matching_part_pairs(RelOptInfo *outer_rel, + RelOptInfo *inner_rel, + PartitionMap *outer_map, + PartitionMap *inner_map, + int nmerged, + List **outer_parts, + List **inner_parts); +static PartitionBoundInfo build_merged_partition_bounds(char strategy, + List *merged_datums, + List *merged_kinds, + List *merged_indexes, + int null_index, + int default_index); +static int get_range_partition(RelOptInfo *rel, + PartitionBoundInfo bi, + int *lb_pos, + PartitionRangeBound *lb, + PartitionRangeBound *ub); +static int get_range_partition_internal(PartitionBoundInfo bi, + int *lb_pos, + PartitionRangeBound *lb, + PartitionRangeBound *ub); +static bool compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + PartitionRangeBound *outer_lb, + PartitionRangeBound *outer_ub, + PartitionRangeBound *inner_lb, + PartitionRangeBound *inner_ub, + int *lb_cmpval, int *ub_cmpval); +static void get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, JoinType jointype, + PartitionRangeBound *outer_lb, + PartitionRangeBound *outer_ub, + PartitionRangeBound *inner_lb, + PartitionRangeBound *inner_ub, + int lb_cmpval, int ub_cmpval, + PartitionRangeBound *merged_lb, + PartitionRangeBound *merged_ub); +static void add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + PartitionRangeBound *merged_lb, + PartitionRangeBound *merged_ub, + int merged_index, + List **merged_datums, + List **merged_kinds, + List **merged_indexes); static PartitionRangeBound *make_one_partition_rbound(PartitionKey key, int index, List *datums, bool lower); static int32 partition_hbound_cmp(int modulus1, int remainder1, int modulus2, @@ -866,6 +996,1714 @@ partition_bounds_copy(PartitionBoundInfo src, return dest; } +/* + * partition_bounds_merge + * + * This function builds and returns the partition bounds for a join relation + * between input relations, creating two lists of partitions, which are + * returned to *outer_parts and *inner_parts respectively. The lists contain + * the same number of partitions, and the partitions at the same positions in + * the lists indicate join pairs used for partitioned join. + * + * This function returns NULL, setting *outer_parts and *inner_parts to NIL, + * if a partition on one side matches multiple partitions on the other side, + * in which case we currently don't support partitioned join. + */ +PartitionBoundInfo +partition_bounds_merge(int partnatts, + FmgrInfo *partsupfunc, Oid *partcollation, + RelOptInfo *outer_rel, RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, List **inner_parts) +{ + PartitionBoundInfo outer_binfo = outer_rel->boundinfo; + PartitionBoundInfo inner_binfo = inner_rel->boundinfo; + char strategy; + + /* + * Currently, this function is called only from try_partitionwise_join(), + * so the join type should be INNER, LEFT, FULL, SEMI, or ANTI. + */ + if (jointype != JOIN_INNER && jointype != JOIN_LEFT && + jointype != JOIN_FULL && jointype != JOIN_SEMI && + jointype != JOIN_ANTI) + elog(ERROR, "unrecognized join type: %d", (int) jointype); + + /* Bail out if the partitioning strategies are different. */ + if (outer_binfo->strategy != inner_binfo->strategy) + return NULL; + + strategy = outer_binfo->strategy; + *outer_parts = *inner_parts = NIL; + switch (strategy) + { + case PARTITION_STRATEGY_HASH: + + /* + * For hash partitioned tables, we currently support partitioned + * join only when the partition bounds for them exactly match. + * + * XXX: it might be possible to relax the restriction to support + * cases where hash partitioned tables have missing partitions + * and/or different moduli, but it's not clear if it would be + * useful to support the former case since it's unusual to have + * missing partitions. On the other hand, it would be useful to + * support the latter case, but in that case, there is a high + * probability that a partition on one side will match multiple + * partitions on the other side, which is the scenario the current + * implementation of partitioned join can't handle. + */ + return NULL; + + case PARTITION_STRATEGY_LIST: + return merge_list_bounds(partsupfunc, + partcollation, + outer_rel, + inner_rel, + jointype, + outer_parts, + inner_parts); + + case PARTITION_STRATEGY_RANGE: + return merge_range_bounds(partnatts, + partsupfunc, + partcollation, + outer_rel, + inner_rel, + jointype, + outer_parts, + inner_parts); + + default: + elog(ERROR, "unexpected partition strategy: %d", (int) strategy); + return NULL; /* keep compiler quiet */ + } +} + +/* + * merge_list_bounds + * Create the partition bounds for a join relation between list + * partitioned tables, if possible + */ +static PartitionBoundInfo +merge_list_bounds(FmgrInfo *partsupfunc, Oid *partcollation, + RelOptInfo *outer_rel, RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, List **inner_parts) +{ + PartitionBoundInfo merged_bounds = NULL; + PartitionBoundInfo outer_bi = outer_rel->boundinfo; + PartitionBoundInfo inner_bi = inner_rel->boundinfo; + bool outer_has_default = partition_bound_has_default(outer_bi); + bool inner_has_default = partition_bound_has_default(inner_bi); + int outer_default = outer_bi->default_index; + int inner_default = inner_bi->default_index; + bool outer_has_null = partition_bound_accepts_nulls(outer_bi); + bool inner_has_null = partition_bound_accepts_nulls(inner_bi); + PartitionMap outer_map; + PartitionMap inner_map; + int outer_pos; + int inner_pos; + int next_index = 0; + int null_index = -1; + int default_index = -1; + List *merged_datums = NIL; + List *merged_indexes = NIL; + + Assert(*outer_parts == NIL); + Assert(*inner_parts == NIL); + Assert(outer_bi->strategy == inner_bi->strategy && + outer_bi->strategy == PARTITION_STRATEGY_LIST); + /* List partitioning doesn't require kinds. */ + Assert(!outer_bi->kind && !inner_bi->kind); + + init_partition_map(outer_rel, &outer_map); + init_partition_map(inner_rel, &inner_map); + + /* + * If the default partitions (if any) have been proven empty, deem them + * non-existent. + */ + if (outer_has_default && is_dummy_partition(outer_rel, outer_default)) + outer_has_default = false; + if (inner_has_default && is_dummy_partition(inner_rel, inner_default)) + inner_has_default = false; + + /* + * Merge partitions from both sides. In each iteration we compare a pair + * of list values, one from each side, and decide whether the corresponding + * partitions match or not. If the two values match exactly, move to the + * next pair of list values, otherwise move to the next list value on the + * side with a smaller list value. + */ + outer_pos = inner_pos = 0; + while (outer_pos < outer_bi->ndatums || inner_pos < inner_bi->ndatums) + { + int outer_index = -1; + int inner_index = -1; + Datum *outer_datums; + Datum *inner_datums; + int cmpval; + Datum *merged_datum = NULL; + int merged_index = -1; + + if (outer_pos < outer_bi->ndatums) + { + /* + * If the partition on the outer side has been proven empty, ignore + * it and move to the next datum on the outer side. + */ + outer_index = outer_bi->indexes[outer_pos]; + if (is_dummy_partition(outer_rel, outer_index)) + { + outer_pos++; + continue; + } + } + if (inner_pos < inner_bi->ndatums) + { + /* + * If the partition on the inner side has been proven empty, ignore + * it and move to the next datum on the inner side. + */ + inner_index = inner_bi->indexes[inner_pos]; + if (is_dummy_partition(inner_rel, inner_index)) + { + inner_pos++; + continue; + } + } + + /* Get the list values. */ + outer_datums = outer_pos < outer_bi->ndatums ? + outer_bi->datums[outer_pos] : NULL; + inner_datums = inner_pos < inner_bi->ndatums ? + inner_bi->datums[inner_pos] : NULL; + + /* + * We run this loop till both sides finish. This allows us to avoid + * duplicating code to handle the remaining values on the side which + * finishes later. For that we set the comparison parameter cmpval in + * such a way that it appears as if the side which finishes earlier has + * an extra value higher than any other value on the unfinished side. + * That way we advance the values on the unfinished side till all of + * its values are exhausted. + */ + if (outer_pos >= outer_bi->ndatums) + cmpval = 1; + else if (inner_pos >= inner_bi->ndatums) + cmpval = -1; + else + { + Assert(outer_datums != NULL && inner_datums != NULL); + cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0], + partcollation[0], + outer_datums[0], + inner_datums[0])); + } + + if (cmpval == 0) + { + /* Two list values match exactly. */ + Assert(outer_pos < outer_bi->ndatums); + Assert(inner_pos < inner_bi->ndatums); + Assert(outer_index >= 0); + Assert(inner_index >= 0); + + /* + * Try merging both paritions. If successful, add the list value + * and index of the merged partition below. + */ + merged_index = merge_matching_partitions(&outer_map, &inner_map, + outer_index, inner_index, + &next_index); + if (merged_index == -1) + goto cleanup; + + merged_datum = outer_datums; + + /* Move to the next pair of list values. */ + outer_pos++; + inner_pos++; + } + else if (cmpval < 0) + { + /* A list value missing from the inner side. */ + Assert(outer_pos < outer_bi->ndatums); + + /* + * If the inner side has the default partition, or this is an outer + * join, try to assign a merged partition to the outer partition + * (see process_outer_partition()). Otherwise, the outer partition + * will not contribute to the result. + */ + if (inner_has_default || IS_OUTER_JOIN(jointype)) + { + /* Get the outer partition. */ + outer_index = outer_bi->indexes[outer_pos]; + Assert(outer_index >= 0); + merged_index = process_outer_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + outer_index, + inner_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + merged_datum = outer_datums; + } + + /* Move to the next list value on the outer side. */ + outer_pos++; + } + else + { + /* A list value missing from the outer side. */ + Assert(cmpval > 0); + Assert(inner_pos < inner_bi->ndatums); + + /* + * If the outer side has the default partition, or this is a FULL + * join, try to assign a merged partition to the inner partition + * (see process_inner_partition()). Otherwise, the inner partition + * will not contribute to the result. + */ + if (outer_has_default || jointype == JOIN_FULL) + { + /* Get the inner partition. */ + inner_index = inner_bi->indexes[inner_pos]; + Assert(inner_index >= 0); + merged_index = process_inner_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + inner_index, + outer_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + merged_datum = inner_datums; + } + + /* Move to the next list value on the inner side. */ + inner_pos++; + } + + /* + * If we assigned a merged partition, add the list value and index of + * the merged partition if appropriate. + */ + if (merged_index >= 0 && merged_index != default_index) + { + merged_datums = lappend(merged_datums, merged_datum); + merged_indexes = lappend_int(merged_indexes, merged_index); + } + } + + /* + * If the NULL partitions (if any) have been proven empty, deem them + * non-existent. + */ + if (outer_has_null && + is_dummy_partition(outer_rel, outer_bi->null_index)) + outer_has_null = false; + if (inner_has_null && + is_dummy_partition(inner_rel, inner_bi->null_index)) + inner_has_null = false; + + /* Merge the NULL partitions if any. */ + if (outer_has_null || inner_has_null) + merge_null_partitions(&outer_map, &inner_map, + outer_has_null, inner_has_null, + outer_bi->null_index, inner_bi->null_index, + jointype, &next_index, &null_index); + else + Assert(null_index == -1); + + /* Merge the default partitions if any. */ + if (outer_has_default || inner_has_default) + merge_default_partitions(&outer_map, &inner_map, + outer_has_default, inner_has_default, + outer_default, inner_default, + jointype, &next_index, &default_index); + else + Assert(default_index == -1); + + /* If we have merged partitions, create the partition bounds. */ + if (next_index > 0) + { + /* Fix the merged_indexes list if necessary. */ + if (outer_map.did_remapping || inner_map.did_remapping) + { + Assert(jointype == JOIN_FULL); + fix_merged_indexes(&outer_map, &inner_map, + next_index, merged_indexes); + } + + /* Use maps to match partitions from inputs. */ + generate_matching_part_pairs(outer_rel, inner_rel, + &outer_map, &inner_map, + next_index, + outer_parts, inner_parts); + Assert(*outer_parts != NIL); + Assert(*inner_parts != NIL); + Assert(list_length(*outer_parts) == list_length(*inner_parts)); + Assert(list_length(*outer_parts) <= next_index); + + /* Make a PartitionBoundInfo struct to return. */ + merged_bounds = build_merged_partition_bounds(outer_bi->strategy, + merged_datums, + NIL, + merged_indexes, + null_index, + default_index); + Assert(merged_bounds); + } + +cleanup: + /* Free local memory before returning. */ + list_free(merged_datums); + list_free(merged_indexes); + free_partition_map(&outer_map); + free_partition_map(&inner_map); + + return merged_bounds; +} + +/* + * merge_range_bounds + * Create the partition bounds for a join relation between range + * partitioned tables, if possible + */ +static PartitionBoundInfo +merge_range_bounds(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + RelOptInfo *outer_rel, RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, List **inner_parts) +{ + PartitionBoundInfo merged_bounds = NULL; + PartitionBoundInfo outer_bi = outer_rel->boundinfo; + PartitionBoundInfo inner_bi = inner_rel->boundinfo; + bool outer_has_default = partition_bound_has_default(outer_bi); + bool inner_has_default = partition_bound_has_default(inner_bi); + int outer_default = outer_bi->default_index; + int inner_default = inner_bi->default_index; + PartitionMap outer_map; + PartitionMap inner_map; + int outer_index; + int inner_index; + int outer_lb_pos; + int inner_lb_pos; + PartitionRangeBound outer_lb; + PartitionRangeBound outer_ub; + PartitionRangeBound inner_lb; + PartitionRangeBound inner_ub; + int next_index = 0; + int default_index = -1; + List *merged_datums = NIL; + List *merged_kinds = NIL; + List *merged_indexes = NIL; + + Assert(*outer_parts == NIL); + Assert(*inner_parts == NIL); + Assert(outer_bi->strategy == inner_bi->strategy && + outer_bi->strategy == PARTITION_STRATEGY_RANGE); + + init_partition_map(outer_rel, &outer_map); + init_partition_map(inner_rel, &inner_map); + + /* + * If the default partitions (if any) have been proven empty, deem them + * non-existent. + */ + if (outer_has_default && is_dummy_partition(outer_rel, outer_default)) + outer_has_default = false; + if (inner_has_default && is_dummy_partition(inner_rel, inner_default)) + inner_has_default = false; + + /* + * Merge partitions from both sides. In each iteration we compare a pair + * of ranges, one from each side, and decide whether the corresponding + * partitions match or not. If the two ranges overlap, move to the next + * pair of ranges, otherwise move to the next range on the side with a + * lower range. outer_lb_pos/inner_lb_pos keep track of the positions of + * lower bounds in the datums arrays in the outer/inner PartitionBoundInfos + * respectively. + */ + outer_lb_pos = inner_lb_pos = 0; + outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos, + &outer_lb, &outer_ub); + inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos, + &inner_lb, &inner_ub); + while (outer_index >= 0 || inner_index >= 0) + { + bool overlap; + int ub_cmpval; + int lb_cmpval; + PartitionRangeBound merged_lb = {-1, NULL, NULL, true}; + PartitionRangeBound merged_ub = {-1, NULL, NULL, false}; + int merged_index = -1; + + /* + * We run this loop till both sides finish. This allows us to avoid + * duplicating code to handle the remaining ranges on the side which + * finishes later. For that we set the comparison parameter cmpval in + * such a way that it appears as if the side which finishes earlier has + * an extra range higher than any other range on the unfinished side. + * That way we advance the ranges on the unfinished side till all of + * its ranges are exhausted. + */ + if (outer_index == -1) + { + overlap = false; + lb_cmpval = 1; + ub_cmpval = 1; + } + else if (inner_index == -1) + { + overlap = false; + lb_cmpval = -1; + ub_cmpval = -1; + } + else + overlap = compare_range_partitions(partnatts, partsupfuncs, + partcollations, + &outer_lb, &outer_ub, + &inner_lb, &inner_ub, + &lb_cmpval, &ub_cmpval); + + if (overlap) + { + /* Two ranges overlap; form a join pair. */ + + PartitionRangeBound save_outer_ub; + PartitionRangeBound save_inner_ub; + + /* Both partitions should not have been merged yet. */ + Assert(outer_index >= 0); + Assert(outer_map.merged_indexes[outer_index] == -1 && + outer_map.merged[outer_index] == false); + Assert(inner_index >= 0); + Assert(inner_map.merged_indexes[inner_index] == -1 && + inner_map.merged[inner_index] == false); + + /* + * Get the index of the merged partition. Both partitions aren't + * merged yet, so the partitions should be merged successfully. + */ + merged_index = merge_matching_partitions(&outer_map, &inner_map, + outer_index, inner_index, + &next_index); + Assert(merged_index >= 0); + + /* Get the range of the merged partition. */ + get_merged_range_bounds(partnatts, partsupfuncs, + partcollations, jointype, + &outer_lb, &outer_ub, + &inner_lb, &inner_ub, + lb_cmpval, ub_cmpval, + &merged_lb, &merged_ub); + + /* Save the upper bounds of both partitions for use below. */ + save_outer_ub = outer_ub; + save_inner_ub = inner_ub; + + /* Move to the next pair of ranges. */ + outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos, + &outer_lb, &outer_ub); + inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos, + &inner_lb, &inner_ub); + + /* + * If the range of a partition on one side overlaps the range of + * the next partition on the other side, that will cause the + * partition on one side to match at least two partitions on the + * other side, which is the case that we currently don't support + * partitioned join for; give up. + */ + if (ub_cmpval > 0 && inner_index >= 0 && + compare_range_bounds(partnatts, partsupfuncs, partcollations, + &save_outer_ub, &inner_lb) > 0) + goto cleanup; + if (ub_cmpval < 0 && outer_index >= 0 && + compare_range_bounds(partnatts, partsupfuncs, partcollations, + &outer_lb, &save_inner_ub) < 0) + goto cleanup; + + /* + * A row from a non-overlapping portion (if any) of a partition + * on one side might find its join partner in the default + * partition (if any) on the other side, causing the same + * situation as above; give up in that case. + */ + if ((outer_has_default && (lb_cmpval > 0 || ub_cmpval < 0)) || + (inner_has_default && (lb_cmpval < 0 || ub_cmpval > 0))) + goto cleanup; + } + else if (ub_cmpval < 0) + { + /* A non-overlapping outer range. */ + + /* The outer partition should not have been merged yet. */ + Assert(outer_index >= 0); + Assert(outer_map.merged_indexes[outer_index] == -1 && + outer_map.merged[outer_index] == false); + + /* + * If the inner side has the default partition, or this is an outer + * join, try to assign a merged partition to the outer partition + * (see process_outer_partition()). Otherwise, the outer partition + * will not contribute to the result. + */ + if (inner_has_default || IS_OUTER_JOIN(jointype)) + { + merged_index = process_outer_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + outer_index, + inner_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + merged_lb = outer_lb; + merged_ub = outer_ub; + } + + /* Move to the next range on the outer side. */ + outer_index = get_range_partition(outer_rel, outer_bi, &outer_lb_pos, + &outer_lb, &outer_ub); + } + else + { + /* A non-overlapping inner range. */ + Assert(ub_cmpval > 0); + + /* The inner partition should not have been merged yet. */ + Assert(inner_index >= 0); + Assert(inner_map.merged_indexes[inner_index] == -1 && + inner_map.merged[inner_index] == false); + + /* + * If the outer side has the default partition, or this is a FULL + * join, try to assign a merged partition to the inner partition + * (see process_inner_partition()). Otherwise, the inner partition + * will not contribute to the result. + */ + if (outer_has_default || jointype == JOIN_FULL) + { + merged_index = process_inner_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + inner_index, + outer_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + merged_lb = inner_lb; + merged_ub = inner_ub; + } + + /* Move to the next range on the inner side. */ + inner_index = get_range_partition(inner_rel, inner_bi, &inner_lb_pos, + &inner_lb, &inner_ub); + } + + /* + * If we assigned a merged partition, add the range bounds and index of + * the merged partition if appropriate. + */ + if (merged_index >= 0 && merged_index != default_index) + add_merged_range_bounds(partnatts, partsupfuncs, partcollations, + &merged_lb, &merged_ub, merged_index, + &merged_datums, &merged_kinds, + &merged_indexes); + } + + /* Merge the default partitions if any. */ + if (outer_has_default || inner_has_default) + merge_default_partitions(&outer_map, &inner_map, + outer_has_default, inner_has_default, + outer_default, inner_default, + jointype, &next_index, &default_index); + else + Assert(default_index == -1); + + /* If we have merged partitions, create the partition bounds. */ + if (next_index > 0) + { + /* + * Unlike the case of list partitioning, we wouldn't have re-merged + * partitions, so did_remapping should be left alone. + */ + Assert(!outer_map.did_remapping); + Assert(!inner_map.did_remapping); + + /* Use maps to match partitions from inputs. */ + generate_matching_part_pairs(outer_rel, inner_rel, + &outer_map, &inner_map, + next_index, + outer_parts, inner_parts); + Assert(*outer_parts != NIL); + Assert(*inner_parts != NIL); + Assert(list_length(*outer_parts) == list_length(*inner_parts)); + Assert(list_length(*outer_parts) == next_index); + + /* Make a PartitionBoundInfo struct to return. */ + merged_bounds = build_merged_partition_bounds(outer_bi->strategy, + merged_datums, + merged_kinds, + merged_indexes, + -1, + default_index); + Assert(merged_bounds); + } + +cleanup: + /* Free local memory before returning. */ + list_free(merged_datums); + list_free(merged_kinds); + list_free(merged_indexes); + free_partition_map(&outer_map); + free_partition_map(&inner_map); + + return merged_bounds; +} + +/* + * init_partition_map + * Initialize a PartitionMap struct for given relation + */ +static void +init_partition_map(RelOptInfo *rel, PartitionMap *map) +{ + int nparts = rel->nparts; + int i; + + map->nparts = nparts; + map->merged_indexes = (int *) palloc(sizeof(int) * nparts); + map->merged = (bool *) palloc(sizeof(bool) * nparts); + map->did_remapping = false; + map->old_indexes = (int *) palloc(sizeof(int) * nparts); + for (i = 0; i < nparts; i++) + { + map->merged_indexes[i] = map->old_indexes[i] = -1; + map->merged[i] = false; + } +} + +/* + * free_partition_map + */ +static void +free_partition_map(PartitionMap *map) +{ + pfree(map->merged_indexes); + pfree(map->merged); + pfree(map->old_indexes); +} + +/* + * is_dummy_partition --- has partition been proven empty? + */ +static bool +is_dummy_partition(RelOptInfo *rel, int part_index) +{ + RelOptInfo *part_rel; + + Assert(part_index >= 0); + part_rel = rel->part_rels[part_index]; + if (part_rel == NULL || IS_DUMMY_REL(part_rel)) + return true; + return false; +} + +/* + * merge_matching_partitions + * Try to merge given outer/inner partitions, and return the index of a + * merged partition produced from them if successful, -1 otherwise + * + * If the merged partition is newly created, *next_index is incremented. + */ +static int +merge_matching_partitions(PartitionMap *outer_map, PartitionMap *inner_map, + int outer_index, int inner_index, int *next_index) +{ + int outer_merged_index; + int inner_merged_index; + bool outer_merged; + bool inner_merged; + + Assert(outer_index >= 0 && outer_index < outer_map->nparts); + outer_merged_index = outer_map->merged_indexes[outer_index]; + outer_merged = outer_map->merged[outer_index]; + Assert(inner_index >= 0 && inner_index < inner_map->nparts); + inner_merged_index = inner_map->merged_indexes[inner_index]; + inner_merged = inner_map->merged[inner_index]; + + /* + * Handle cases where we have already assigned a merged partition to each + * of the given partitions. + */ + if (outer_merged_index >= 0 && inner_merged_index >= 0) + { + /* + * If the mereged partitions are the same, no need to do anything; + * return the index of the merged partitions. Otherwise, if each of + * the given partitions has been merged with a dummy partition on the + * other side, re-map them to either of the two merged partitions. + * Otherwise, they can't be merged, so return -1. + */ + if (outer_merged_index == inner_merged_index) + { + Assert(outer_merged); + Assert(inner_merged); + return outer_merged_index; + } + if (!outer_merged && !inner_merged) + { + /* + * This can only happen for a list-partitioning case. We re-map + * them to the merged partition with the smaller of the two merged + * indexes to preserve the property that the canonical order of + * list partitions is determined by the indexes assigned to the + * smallest list value of each partition. + */ + if (outer_merged_index < inner_merged_index) + { + outer_map->merged[outer_index] = true; + inner_map->merged_indexes[inner_index] = outer_merged_index; + inner_map->merged[inner_index] = true; + inner_map->did_remapping = true; + inner_map->old_indexes[inner_index] = inner_merged_index; + return outer_merged_index; + } + else + { + inner_map->merged[inner_index] = true; + outer_map->merged_indexes[outer_index] = inner_merged_index; + outer_map->merged[outer_index] = true; + outer_map->did_remapping = true; + outer_map->old_indexes[outer_index] = outer_merged_index; + return inner_merged_index; + } + } + return -1; + } + + /* At least one of the given partitions should not have yet been merged. */ + Assert(outer_merged_index == -1 || inner_merged_index == -1); + + /* + * If neither of them has been merged, merge them. Otherwise, if one has + * been merged with a dummy relation on the other side (and the other + * hasn't yet been merged with anything), re-merge them. Otherwise, they + * can't be merged, so return -1. + */ + if (outer_merged_index == -1 && inner_merged_index == -1) + { + int merged_index = *next_index; + + Assert(!outer_merged); + Assert(!inner_merged); + outer_map->merged_indexes[outer_index] = merged_index; + outer_map->merged[outer_index] = true; + inner_map->merged_indexes[inner_index] = merged_index; + inner_map->merged[inner_index] = true; + *next_index = *next_index + 1; + return merged_index; + } + if (outer_merged_index >= 0 && !outer_map->merged[outer_index]) + { + Assert(inner_merged_index == -1); + Assert(!inner_merged); + inner_map->merged_indexes[inner_index] = outer_merged_index; + inner_map->merged[inner_index] = true; + outer_map->merged[outer_index] = true; + return outer_merged_index; + } + if (inner_merged_index >= 0 && !inner_map->merged[inner_index]) + { + Assert(outer_merged_index == -1); + Assert(!outer_merged); + outer_map->merged_indexes[outer_index] = inner_merged_index; + outer_map->merged[outer_index] = true; + inner_map->merged[inner_index] = true; + return inner_merged_index; + } + return -1; +} + +/* + * process_outer_partition + * Try to assign given outer partition a merged partition, and return the + * index of the merged partition if successful, -1 otherwise + * + * If the partition is newly created, *next_index is incremented. Also, if it + * is the default partition of the join relation, *default_partition is set to + * the index if not already done. + */ +static int +process_outer_partition(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int outer_index, + int inner_default, + JoinType jointype, + int *next_index, + int *default_index) +{ + int merged_index = -1; + + Assert(outer_index >= 0); + + /* + * If the inner side has the default partition, a row from the outer + * partition might find its join partner in the default partition; try + * merging the outer partition with the default partition. Otherwise, this + * should be an outer join, in which case the outer partition has to be + * scanned all the way anyway; merge the outer partition with a dummy + * partition on the other side. + */ + if (inner_has_default) + { + Assert(inner_default >= 0); + + /* + * If the outer side has the default partition as well, the default + * partition on the inner side will have two matching partitions on the + * other side: the outer partition and the default partition on the + * outer side. Partitionwise join doesn't handle this scenario yet. + */ + if (outer_has_default) + return -1; + + merged_index = merge_matching_partitions(outer_map, inner_map, + outer_index, inner_default, + next_index); + if (merged_index == -1) + return -1; + + /* + * If this is a FULL join, the default partition on the inner side + * has to be scanned all the way anyway, so the resulting partition + * will contain all key values from the default partition, which any + * other partition of the join relation will not contain. Thus the + * resutling partition will act as the default partition of the join + * relation; record the index in *default_index if not already done. + */ + if (jointype == JOIN_FULL) + { + if (*default_index == -1) + *default_index = merged_index; + else + Assert(*default_index == merged_index); + } + } + else + { + Assert(IS_OUTER_JOIN(jointype)); + Assert(jointype != JOIN_RIGHT); + + /* If we have already assigned a partition, no need to do anything. */ + merged_index = outer_map->merged_indexes[outer_index]; + if (merged_index == -1) + merged_index = merge_partition_with_dummy(outer_map, outer_index, + next_index); + } + return merged_index; +} + +/* + * process_inner_partition + * Try to assign given inner partition a merged partition, and return the + * index of the merged partition if successful, -1 otherwise + * + * If the partition is newly created, *next_index is incremented. Also, if it + * is the default partition of the join relation, *default_partition is set to + * the index if not already done. + */ +static int +process_inner_partition(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int inner_index, + int outer_default, + JoinType jointype, + int *next_index, + int *default_index) +{ + int merged_index = -1; + + Assert(inner_index >= 0); + + /* + * If the outer side has the default partition, a row from the inner + * partition might find its join partner in the default partition; try + * merging the inner partition with the default partition. Otherwise, this + * should be a FULL join, in which case the inner partition has to be + * scanned all the way anyway; merge the inner partition with a dummy + * partition on the other side. + */ + if (outer_has_default) + { + Assert(outer_default >= 0); + + /* + * If the inner side has the default partition as well, the default + * partition on the outer side will have two matching partitions on the + * other side: the inner partition and the default partition on the + * inner side. Partitionwise join doesn't handle this scenario yet. + */ + if (inner_has_default) + return -1; + + merged_index = merge_matching_partitions(outer_map, inner_map, + outer_default, inner_index, + next_index); + if (merged_index == -1) + return -1; + + /* + * If this is an outer join, the default partition on the outer side + * has to be scanned all the way anyway, so the resulting partition + * will contain all key values from the default partition, which any + * other partition of the join relation will not contain. Thus the + * resutling partition will act as the default partition of the join + * relation; record the index in *default_index if not already done. + */ + if (IS_OUTER_JOIN(jointype)) + { + Assert(jointype != JOIN_RIGHT); + if (*default_index == -1) + *default_index = merged_index; + else + Assert(*default_index == merged_index); + } + } + else + { + Assert(jointype == JOIN_FULL); + + /* If we have already assigned a partition, no need to do anything. */ + merged_index = inner_map->merged_indexes[inner_index]; + if (merged_index == -1) + merged_index = merge_partition_with_dummy(inner_map, inner_index, + next_index); + } + return merged_index; +} + +/* + * merge_null_partitions + * Merge the NULL partitions from a join's outer and inner sides. + * + * If the merged partition produced from them is the NULL partition of the join + * relation, *null_index is set to the index of the merged partition. + * + * Note: We assume here that the join clause for a partitioned join is strict + * because have_partkey_equi_join() requires that the corresponding operator + * be mergejoinable, and we currently assume that mergejoinable operators are + * strict (see MJEvalOuterValues()/MJEvalInnerValues()). + */ +static void +merge_null_partitions(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_null, + bool inner_has_null, + int outer_null, + int inner_null, + JoinType jointype, + int *next_index, + int *null_index) +{ + bool consider_outer_null = false; + bool consider_inner_null = false; + + Assert(outer_has_null || inner_has_null); + Assert(*null_index == -1); + + /* + * Check whether the NULL partitions have already been merged and if so, + * set the consider_outer_null/consider_inner_null flags. + */ + if (outer_has_null) + { + Assert(outer_null >= 0 && outer_null < outer_map->nparts); + if (outer_map->merged_indexes[outer_null] == -1) + consider_outer_null = true; + } + if (inner_has_null) + { + Assert(inner_null >= 0 && inner_null < inner_map->nparts); + if (inner_map->merged_indexes[inner_null] == -1) + consider_inner_null = true; + } + + /* If both flags are set false, we don't need to do anything. */ + if (!consider_outer_null && !consider_inner_null) + return; + + if (consider_outer_null && !consider_inner_null) + { + Assert(outer_has_null); + + /* + * If this is an outer join, the NULL partition on the outer side has + * to be scanned all the way anyway; merge the NULL partition with a + * dummy partition on the other side. In that case consider_outer_null + * means that the NULL partition only contains NULL values as the key + * values, so the merged partition will do so; treat it as the NULL + * partition of the join relation. + */ + if (IS_OUTER_JOIN(jointype)) + { + Assert(jointype != JOIN_RIGHT); + *null_index = merge_partition_with_dummy(outer_map, outer_null, + next_index); + } + } + else if (!consider_outer_null && consider_inner_null) + { + Assert(inner_has_null); + + /* + * If this is a FULL join, the NULL partition on the inner side has + * to be scanned all the way anyway; merge the NULL partition with a + * dummy partition on the other side. In that case consider_inner_null + * means that the NULL partition only contains NULL values as the key + * values, so the merged partition will do so; treat it as the NULL + * partition of the join relation. + */ + if (jointype == JOIN_FULL) + *null_index = merge_partition_with_dummy(inner_map, inner_null, + next_index); + } + else + { + Assert(consider_outer_null && consider_inner_null); + Assert(outer_has_null); + Assert(inner_has_null); + + /* + * If this is an outer join, the NULL partition on the outer side (and + * that on the inner side if this is a FULL join) have to be scanned + * all the way anyway, so merge them. Note that each of the NULL + * partitions isn't merged yet, so they should be merged successfully. + * Like the above, each of the NULL partitions only contains NULL + * values as the key values, so the merged partition will do so; treat + * it as the NULL partition of the join relation. + * + * Note: if this an INNER/SEMI join, the join clause will never be + * satisfied by two NULL values (see comments above), so both the NULL + * partitions can be eliminated. + */ + if (IS_OUTER_JOIN(jointype)) + { + Assert(jointype != JOIN_RIGHT); + *null_index = merge_matching_partitions(outer_map, inner_map, + outer_null, inner_null, + next_index); + Assert(*null_index >= 0); + } + } +} + +/* + * merge_default_partitions + * Merge the default partitions from a join's outer and inner sides. + * + * If the merged partition produced from them is the default partition of the + * join relation, *default_index is set to the index of the merged partition. + */ +static void +merge_default_partitions(PartitionMap *outer_map, + PartitionMap *inner_map, + bool outer_has_default, + bool inner_has_default, + int outer_default, + int inner_default, + JoinType jointype, + int *next_index, + int *default_index) +{ + int outer_merged_index = -1; + int inner_merged_index = -1; + + Assert(outer_has_default || inner_has_default); + + /* Get the merged partition indexes for the default partitions. */ + if (outer_has_default) + { + Assert(outer_default >= 0 && outer_default < outer_map->nparts); + outer_merged_index = outer_map->merged_indexes[outer_default]; + } + if (inner_has_default) + { + Assert(inner_default >= 0 && inner_default < inner_map->nparts); + inner_merged_index = inner_map->merged_indexes[inner_default]; + } + + if (outer_has_default && !inner_has_default) + { + /* + * If this is an outer join, the default partition on the outer side + * has to be scanned all the way anyway; if we have not yet assigned a + * partition, merge the default partition with a dummy partition on the + * other side. The merged partition will act as the default partition + * of the join relation (see comments in process_inner_partition()). + */ + if (IS_OUTER_JOIN(jointype)) + { + Assert(jointype != JOIN_RIGHT); + if (outer_merged_index == -1) + { + Assert(*default_index == -1); + *default_index = merge_partition_with_dummy(outer_map, + outer_default, + next_index); + } + else + Assert(*default_index == outer_merged_index); + } + else + Assert(*default_index == -1); + } + else if (!outer_has_default && inner_has_default) + { + /* + * If this is a FULL join, the default partition on the inner side + * has to be scanned all the way anyway; if we have not yet assigned a + * partition, merge the default partition with a dummy partition on the + * other side. The merged partition will act as the default partition + * of the join relation (see comments in process_outer_partition()). + */ + if (jointype == JOIN_FULL) + { + if (inner_merged_index == -1) + { + Assert(*default_index == -1); + *default_index = merge_partition_with_dummy(inner_map, + inner_default, + next_index); + } + else + Assert(*default_index == inner_merged_index); + } + else + Assert(*default_index == -1); + } + else + { + Assert(outer_has_default && inner_has_default); + + /* + * The default partitions have to be joined with each other, so merge + * them. Note that each of the default partitions isn't merged yet + * (see, process_outer_partition()/process_innerer_partition()), so + * they should be merged successfully. The merged partition will act + * as the default partition of the join relation. + */ + Assert(outer_merged_index == -1); + Assert(inner_merged_index == -1); + Assert(*default_index == -1); + *default_index = merge_matching_partitions(outer_map, + inner_map, + outer_default, + inner_default, + next_index); + Assert(*default_index >= 0); + } +} + +/* + * merge_partition_with_dummy + * Assign given partition a new partition of a join relation + * + * Note: The caller assumes that the given partition doesn't have a non-dummy + * matching partition on the other side, but if the given partition finds the + * matchig partition later, we will adjust the assignment. + */ +static int +merge_partition_with_dummy(PartitionMap *map, int index, int *next_index) +{ + int merged_index = *next_index; + + Assert(index >= 0 && index < map->nparts); + Assert(map->merged_indexes[index] == -1); + Assert(!map->merged[index]); + map->merged_indexes[index] = merged_index; + /* Leave the merged flag alone! */ + *next_index = *next_index + 1; + return merged_index; +} + +/* + * fix_merged_indexes + * Adjust merged indexes of re-merged partitions + */ +static void +fix_merged_indexes(PartitionMap *outer_map, PartitionMap *inner_map, + int nmerged, List *merged_indexes) +{ + int *new_indexes; + int merged_index; + int i; + ListCell *lc; + + Assert(nmerged > 0); + + new_indexes = (int *) palloc(sizeof(int) * nmerged); + for (i = 0; i < nmerged; i++) + new_indexes[i] = -1; + + /* Build the mapping of old merged indexes to new merged indexes. */ + if (outer_map->did_remapping) + { + for (i = 0; i < outer_map->nparts; i++) + { + merged_index = outer_map->old_indexes[i]; + if (merged_index >= 0) + new_indexes[merged_index] = outer_map->merged_indexes[i]; + } + } + if (inner_map->did_remapping) + { + for (i = 0; i < inner_map->nparts; i++) + { + merged_index = inner_map->old_indexes[i]; + if (merged_index >= 0) + new_indexes[merged_index] = inner_map->merged_indexes[i]; + } + } + + /* Fix the merged_indexes list using the mapping. */ + foreach(lc, merged_indexes) + { + merged_index = lfirst_int(lc); + Assert(merged_index >= 0); + if (new_indexes[merged_index] >= 0) + lfirst_int(lc) = new_indexes[merged_index]; + } + + pfree(new_indexes); +} + +/* + * generate_matching_part_pairs + * Generate a pair of lists of partitions that produce merged partitions + * + * The lists of partitions are built in the order of merged partition indexes, + * and returned in *outer_parts and *inner_parts. + */ +static void +generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel, + PartitionMap *outer_map, PartitionMap *inner_map, + int nmerged, + List **outer_parts, List **inner_parts) +{ + int outer_nparts = outer_map->nparts; + int inner_nparts = inner_map->nparts; + int *outer_indexes; + int *inner_indexes; + int max_nparts; + int i; + + Assert(nmerged > 0); + Assert(*outer_parts == NIL); + Assert(*inner_parts == NIL); + + outer_indexes = (int *) palloc(sizeof(int) * nmerged); + inner_indexes = (int *) palloc(sizeof(int) * nmerged); + for (i = 0; i < nmerged; i++) + outer_indexes[i] = inner_indexes[i] = -1; + + /* Set pairs of matching partitions. */ + Assert(outer_nparts == outer_rel->nparts); + Assert(inner_nparts == inner_rel->nparts); + max_nparts = Max(outer_nparts, inner_nparts); + for (i = 0; i < max_nparts; i++) + { + if (i < outer_nparts) + { + int merged_index = outer_map->merged_indexes[i]; + + if (merged_index >= 0) + { + Assert(merged_index < nmerged); + outer_indexes[merged_index] = i; + } + } + if (i < inner_nparts) + { + int merged_index = inner_map->merged_indexes[i]; + + if (merged_index >= 0) + { + Assert(merged_index < nmerged); + inner_indexes[merged_index] = i; + } + } + } + + /* Build the list pairs. */ + for (i = 0; i < nmerged; i++) + { + int outer_index = outer_indexes[i]; + int inner_index = inner_indexes[i]; + + /* + * If both partitions are dummy, it means the merged partition that had + * been assigned to the outer/inner partition was removed when + * re-merging the outer/inner partition in merge_matching_partitions(); + * ignore the merged partition. + */ + if (outer_index == -1 && inner_index == -1) + continue; + + *outer_parts = lappend(*outer_parts, outer_index >= 0 ? + outer_rel->part_rels[outer_index] : NULL); + *inner_parts = lappend(*inner_parts, inner_index >= 0 ? + inner_rel->part_rels[inner_index] : NULL); + } + + pfree(outer_indexes); + pfree(inner_indexes); +} + +/* + * build_merged_partition_bounds + * Create a PartitionBoundInfo struct from merged partition bounds + */ +static PartitionBoundInfo +build_merged_partition_bounds(char strategy, List *merged_datums, + List *merged_kinds, List *merged_indexes, + int null_index, int default_index) +{ + PartitionBoundInfo merged_bounds; + int ndatums = list_length(merged_datums); + int pos; + ListCell *lc; + + merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData)); + merged_bounds->strategy = strategy; + merged_bounds->ndatums = ndatums; + + merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * ndatums); + pos = 0; + foreach(lc, merged_datums) + merged_bounds->datums[pos++] = (Datum *) lfirst(lc); + + if (strategy == PARTITION_STRATEGY_RANGE) + { + Assert(list_length(merged_kinds) == ndatums); + merged_bounds->kind = (PartitionRangeDatumKind **) + palloc(sizeof(PartitionRangeDatumKind *) * ndatums); + pos = 0; + foreach(lc, merged_kinds) + merged_bounds->kind[pos++] = (PartitionRangeDatumKind *) lfirst(lc); + + /* There are ndatums+1 indexes in the case of range partitioning. */ + merged_indexes = lappend_int(merged_indexes, -1); + ndatums++; + } + else + { + Assert(strategy == PARTITION_STRATEGY_LIST); + Assert(merged_kinds == NIL); + merged_bounds->kind = NULL; + } + + Assert(list_length(merged_indexes) == ndatums); + merged_bounds->indexes = (int *) palloc(sizeof(int) * ndatums); + pos = 0; + foreach(lc, merged_indexes) + merged_bounds->indexes[pos++] = lfirst_int(lc); + + merged_bounds->null_index = null_index; + merged_bounds->default_index = default_index; + + return merged_bounds; +} + +/* + * get_range_partition + * Get the next non-dummy partition of a range-partitioned relation, + * returning the index of that partition + * + * *lb and *ub are set to the lower and upper bounds of that partition + * respectively, and *lb_pos is advanced to the next lower bound, if any. + */ +static int +get_range_partition(RelOptInfo *rel, + PartitionBoundInfo bi, + int *lb_pos, + PartitionRangeBound *lb, + PartitionRangeBound *ub) +{ + int part_index; + + Assert(bi->strategy == PARTITION_STRATEGY_RANGE); + + do { + part_index = get_range_partition_internal(bi, lb_pos, lb, ub); + if (part_index == -1) + return -1; + } while (is_dummy_partition(rel, part_index)); + + return part_index; +} + +static int +get_range_partition_internal(PartitionBoundInfo bi, + int *lb_pos, + PartitionRangeBound *lb, + PartitionRangeBound *ub) +{ + /* Return the index as -1 if we've exhausted all lower bounds. */ + if (*lb_pos >= bi->ndatums) + return -1; + + /* A lower bound should have at least one more bound after it. */ + Assert(*lb_pos + 1 < bi->ndatums); + + /* Set the lower bound. */ + lb->index = bi->indexes[*lb_pos]; + lb->datums = bi->datums[*lb_pos]; + lb->kind = bi->kind[*lb_pos]; + lb->lower = true; + /* Set the upper bound. */ + ub->index = bi->indexes[*lb_pos + 1]; + ub->datums = bi->datums[*lb_pos + 1]; + ub->kind = bi->kind[*lb_pos + 1]; + ub->lower = false; + + /* The index assigned to an upper bound should be valid. */ + Assert(ub->index >= 0); + + /* + * Advance the position to the next lower bound. If there are no bounds + * left beyond the upper bound, we have reached the last lower bound. + */ + if (*lb_pos + 2 >= bi->ndatums) + *lb_pos = bi->ndatums; + else + { + /* + * If the index assigned to the bound next to the upper bound isn't + * valid, that is the next lower bound; else, the upper bound is also + * the lower bound of the next range partition. + */ + if (bi->indexes[*lb_pos + 2] < 0) + *lb_pos = *lb_pos + 2; + else + *lb_pos = *lb_pos + 1; + } + + return ub->index; +} + +/* + * compare_range_partitions + * Compare the bounds of two range partitions, and return true if the + * two partitions overlap, false otherwise + * + * *lb_cmpval is set to -1, 0, or 1 if the outer partition's lower bound is + * lower than, equal to, or higher than the inner partition's lower bound + * respectively. Likewise, *ub_cmpval is set to -1, 0, or 1 if the outer + * partition's upper bound is lower than, equal to, or higher than the inner + * partition's upper bound respectively. + */ +static bool +compare_range_partitions(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + PartitionRangeBound *outer_lb, + PartitionRangeBound *outer_ub, + PartitionRangeBound *inner_lb, + PartitionRangeBound *inner_ub, + int *lb_cmpval, int *ub_cmpval) +{ + /* + * Check if the outer partition's upper bound is lower than the inner + * partition's lower bound; if so the partitions aren't overlapping. + */ + if (compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_ub, inner_lb) < 0) + { + *lb_cmpval = -1; + *ub_cmpval = -1; + return false; + } + + /* + * Check if the outer partition's lower bound is higher than the inner + * partition's upper bound; if so the partitions aren't overlapping. + */ + if (compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_lb, inner_ub) > 0) + { + *lb_cmpval = 1; + *ub_cmpval = 1; + return false; + } + + /* All other cases indicate overlapping partitions. */ + *lb_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_lb, inner_lb); + *ub_cmpval = compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_ub, inner_ub); + return true; +} + +/* + * get_merged_range_bounds + * Given the bounds of range partitions to be joined, determine the bounds + * of a merged partition produced from the range partitions + * + * *merged_lb and *merged_ub are set to the lower and upper bounds of the + * merged partition. + */ +static void +get_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, JoinType jointype, + PartitionRangeBound *outer_lb, + PartitionRangeBound *outer_ub, + PartitionRangeBound *inner_lb, + PartitionRangeBound *inner_ub, + int lb_cmpval, int ub_cmpval, + PartitionRangeBound *merged_lb, + PartitionRangeBound *merged_ub) +{ + Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_lb, inner_lb) == lb_cmpval); + Assert(compare_range_bounds(partnatts, partsupfuncs, partcollations, + outer_ub, inner_ub) == ub_cmpval); + + switch (jointype) + { + case JOIN_INNER: + case JOIN_SEMI: + + /* + * An INNER/SEMI join will have the rows that fit both sides, so + * the lower bound of the merged partition will be the higher of + * the two lower bounds, and the upper bound of the merged + * partition will be the lower of the two upper bounds. + */ + *merged_lb = (lb_cmpval > 0) ? *outer_lb : *inner_lb; + *merged_ub = (ub_cmpval < 0) ? *outer_ub : *inner_ub; + break; + + case JOIN_LEFT: + case JOIN_ANTI: + + /* + * A LEFT/ANTI join will have all the rows from the outer side, so + * the bounds of the merged partition will be the same as the outer + * bounds. + */ + *merged_lb = *outer_lb; + *merged_ub = *outer_ub; + break; + + case JOIN_FULL: + + /* + * A FULL join will have all the rows from both sides, so the lower + * bound of the merged partition will be the lower of the two lower + * bounds, and the upper bound of the merged partition will be the + * higher of the two upper bounds. + */ + *merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb; + *merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub; + break; + + default: + elog(ERROR, "unrecognized join type: %d", (int) jointype); + } +} + +/* + * add_merged_range_bounds + * Add the bounds of a merged partition to the lists of range bounds + */ +static void +add_merged_range_bounds(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + PartitionRangeBound *merged_lb, + PartitionRangeBound *merged_ub, + int merged_index, + List **merged_datums, + List **merged_kinds, + List **merged_indexes) +{ + int cmpval; + + if (!*merged_datums) + { + /* First merged partition */ + Assert(!*merged_kinds); + Assert(!*merged_indexes); + cmpval = 1; + } + else + { + PartitionRangeBound prev_ub; + + Assert(*merged_datums); + Assert(*merged_kinds); + Assert(*merged_indexes); + + /* Get the last upper bound. */ + prev_ub.index = llast_int(*merged_indexes); + prev_ub.datums = (Datum *) llast(*merged_datums); + prev_ub.kind = (PartitionRangeDatumKind *) llast(*merged_kinds); + prev_ub.lower = false; + + /* + * We pass to partition_rbound_cmp() lower1 as false to prevent it + * from considering the last upper bound to be smaller than the lower + * bound of the merged partition when the values of the two range + * bounds compare equal. + */ + cmpval = partition_rbound_cmp(partnatts, partsupfuncs, partcollations, + merged_lb->datums, merged_lb->kind, + false, &prev_ub); + Assert(cmpval >= 0); + } + + /* + * If the lower bound is higher than the last upper bound, add the lower + * bound with the index as -1 indicating that that is a lower bound; else, + * the last upper bound will be reused as the lower bound of the merged + * partition, so skip this. + */ + if (cmpval > 0) + { + *merged_datums = lappend(*merged_datums, merged_lb->datums); + *merged_kinds = lappend(*merged_kinds, merged_lb->kind); + *merged_indexes = lappend_int(*merged_indexes, -1); + } + + /* Add the upper bound and index of the merged partition. */ + *merged_datums = lappend(*merged_datums, merged_ub->datums); + *merged_kinds = lappend(*merged_kinds, merged_ub->kind); + *merged_indexes = lappend_int(*merged_indexes, merged_index); +} + /* * partitions_are_ordered * Determine whether the partitions described by 'boundinfo' are ordered, diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h index 0ceb809644..622ea2bf63 100644 --- a/src/include/nodes/pathnodes.h +++ b/src/include/nodes/pathnodes.h @@ -579,8 +579,10 @@ typedef struct PartitionSchemeData *PartitionScheme; * part_scheme - Partitioning scheme of the relation * nparts - Number of partitions * boundinfo - Partition bounds + * merged - true if partition bounds are merged ones * partition_qual - Partition constraint if not the root * part_rels - RelOptInfos for each partition + * all_partrels - Relids set of all partition relids * partexprs, nullable_partexprs - Partition key expressions * partitioned_child_rels - RT indexes of unpruned partitions of * this relation that are partitioned tables @@ -718,11 +720,15 @@ typedef struct RelOptInfo /* used for partitioned relations */ PartitionScheme part_scheme; /* Partitioning scheme. */ - int nparts; /* number of partitions */ + int nparts; /* number of partitions; 0 = not partitioned; + * -1 = not yet set */ struct PartitionBoundInfoData *boundinfo; /* Partition bounds */ + bool merged; /* true if partition bounds were created by + * partition_bounds_merge() */ List *partition_qual; /* partition constraint */ struct RelOptInfo **part_rels; /* Array of RelOptInfos of partitions, * stored in the same order of bounds */ + Relids all_partrels; /* Relids set of all partition relids */ List **partexprs; /* Non-nullable partition key expressions. */ List **nullable_partexprs; /* Nullable partition key expressions. */ List *partitioned_child_rels; /* List of RT indexes. */ diff --git a/src/include/partitioning/partbounds.h b/src/include/partitioning/partbounds.h index be940b5192..dfc720720b 100644 --- a/src/include/partitioning/partbounds.h +++ b/src/include/partitioning/partbounds.h @@ -16,6 +16,7 @@ #include "nodes/pg_list.h" #include "partitioning/partdefs.h" #include "utils/relcache.h" +struct RelOptInfo; /* avoid including pathnodes.h here */ /* @@ -87,6 +88,14 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen, PartitionBoundInfo b2); extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src, PartitionKey key); +extern PartitionBoundInfo partition_bounds_merge(int partnatts, + FmgrInfo *partsupfunc, + Oid *partcollation, + struct RelOptInfo *outer_rel, + struct RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, + List **inner_parts); extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts); extern void check_new_partition_bound(char *relname, Relation parent, PartitionBoundSpec *spec); diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out index b3fbe47bde..27588c883e 100644 --- a/src/test/regress/expected/partition_join.out +++ b/src/test/regress/expected/partition_join.out @@ -2055,30 +2055,2606 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c); -> Seq Scan on prt1_n_p2 t1_2 (10 rows) --- partitionwise join can not be applied if only one of joining tables has --- default partition -ALTER TABLE prt2 DETACH PARTITION prt2_p3; -ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600); -ANALYZE prt2; +-- +-- Test advanced partition-matching algorithm for partitioned join +-- +-- Tests for range-partitioned tables +CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_adv_a_idx ON prt1_adv (a); +INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_adv; +CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150); +CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500); +CREATE INDEX prt2_adv_b_idx ON prt2_adv (b); +INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i; +INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i; +INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_adv; +-- inner join EXPLAIN (COSTS OFF) -SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b; - QUERY PLAN --------------------------------------------------- +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ Sort Sort Key: t1.a - -> Hash Join - Hash Cond: (t2.b = t1.a) + -> Append + -> Hash Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 + 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 +(8 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Semi Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash Semi Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash Semi Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + a | b | c +-----+---+------ + 100 | 0 | 0100 + 125 | 0 | 0125 + 200 | 0 | 0200 + 225 | 0 | 0225 + 250 | 0 | 0250 + 275 | 0 | 0275 + 350 | 0 | 0350 + 375 | 0 | 0375 +(8 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a, t2.b + -> Append + -> Hash Right Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 150 | 0150 | | + 175 | 0175 | | + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 + 300 | 0300 | | + 325 | 0325 | | + 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 +(12 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Anti Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash Anti Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash Anti Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + a | b | c +-----+---+------ + 150 | 0 | 0150 + 175 | 0 | 0175 + 300 | 0 | 0300 + 325 | 0 | 0325 +(4 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + QUERY PLAN +-------------------------------------------------------------------------- + Sort + Sort Key: prt1_adv.a, prt2_adv.b + -> Append + -> Hash Full Join + Hash Cond: (prt1_adv_1.a = prt2_adv_1.b) + Filter: (((175) = prt1_adv_1.a) OR ((425) = prt2_adv_1.b)) + -> Seq Scan on prt1_adv_p1 prt1_adv_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p1 prt2_adv_1 + Filter: (a = 0) + -> Hash Full Join + Hash Cond: (prt1_adv_2.a = prt2_adv_2.b) + Filter: (((175) = prt1_adv_2.a) OR ((425) = prt2_adv_2.b)) + -> Seq Scan on prt1_adv_p2 prt1_adv_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p2 prt2_adv_2 + Filter: (a = 0) + -> Hash Full Join + Hash Cond: (prt2_adv_3.b = prt1_adv_3.a) + Filter: (((175) = prt1_adv_3.a) OR ((425) = prt2_adv_3.b)) + -> Seq Scan on prt2_adv_p3 prt2_adv_3 + Filter: (a = 0) + -> Hash + -> Seq Scan on prt1_adv_p3 prt1_adv_3 + Filter: (b = 0) +(27 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 175 | 0175 | | + | | 425 | 0425 +(2 rows) + +-- Test cases where one side has an extra partition +CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i; +ANALYZE prt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 + 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 +(8 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Semi Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash Semi Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash Semi Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + a | b | c +-----+---+------ + 100 | 0 | 0100 + 125 | 0 | 0125 + 200 | 0 | 0200 + 225 | 0 | 0225 + 250 | 0 | 0250 + 275 | 0 | 0275 + 350 | 0 | 0350 + 375 | 0 | 0375 +(8 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a, t2.b + -> Append + -> Hash Right Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 150 | 0150 | | + 175 | 0175 | | + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 + 300 | 0300 | | + 325 | 0325 | | + 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 +(12 rows) + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a; + QUERY PLAN +--------------------------------------------------------- + Sort + Sort Key: t1.b, t2.a + -> Hash Right Join + Hash Cond: (t2.a = t1.b) -> Append - -> Seq Scan on prt2_p1 t2_1 - -> Seq Scan on prt2_p2 t2_2 - -> Seq Scan on prt2_p3 t2_3 + -> Seq Scan on prt1_adv_p1 t2_1 + -> Seq Scan on prt1_adv_p2 t2_2 + -> Seq Scan on prt1_adv_p3 t2_3 -> Hash -> Append - -> Seq Scan on prt1_p1 t1_1 + -> Seq Scan on prt2_adv_p1 t1_1 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p2 t1_2 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p3 t1_3 + Filter: (a = 0) + -> Seq Scan on prt2_adv_extra t1_4 + Filter: (a = 0) +(18 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Anti Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash Anti Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash Anti Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_adv_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + a | b | c +-----+---+------ + 150 | 0 | 0150 + 175 | 0 | 0175 + 300 | 0 | 0300 + 325 | 0 | 0325 +(4 rows) + +-- anti join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.b + -> Hash Anti Join + Hash Cond: (t1.b = t2.a) + -> Append + -> Seq Scan on prt2_adv_p1 t1_1 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p2 t1_2 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p3 t1_3 + Filter: (a = 0) + -> Seq Scan on prt2_adv_extra t1_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_adv_p1 t2_1 + -> Seq Scan on prt1_adv_p2 t2_2 + -> Seq Scan on prt1_adv_p3 t2_3 +(18 rows) + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + QUERY PLAN +---------------------------------------------------------------- + Sort + Sort Key: prt1_adv.a, prt2_adv.b + -> Hash Full Join + Hash Cond: (prt2_adv.b = prt1_adv.a) + Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b)) + -> Append + -> Seq Scan on prt2_adv_p1 prt2_adv_1 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p2 prt2_adv_2 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p3 prt2_adv_3 + Filter: (a = 0) + -> Seq Scan on prt2_adv_extra prt2_adv_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_adv_p1 prt1_adv_1 Filter: (b = 0) - -> Seq Scan on prt1_p2 t1_2 + -> Seq Scan on prt1_adv_p2 prt1_adv_2 Filter: (b = 0) - -> Seq Scan on prt1_p3 t1_3 + -> Seq Scan on prt1_adv_p3 prt1_adv_3 Filter: (b = 0) +(22 rows) + +-- 3-way join where not every pair of relations can do partitioned join +EXPLAIN (COSTS OFF) +SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a; + QUERY PLAN +-------------------------------------------------------------------------------- + Sort + Sort Key: t1.b, t2.a + -> Append + -> Nested Loop Left Join + -> Nested Loop + -> Seq Scan on prt2_adv_p1 t1_1 + Filter: (a = 0) + -> Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t3_1 + Index Cond: (a = t1_1.b) + -> Index Scan using prt1_adv_p1_a_idx on prt1_adv_p1 t2_1 + Index Cond: (a = t1_1.b) + -> Hash Right Join + Hash Cond: (t2_2.a = t1_2.b) + -> Seq Scan on prt1_adv_p2 t2_2 + -> Hash + -> Hash Join + Hash Cond: (t3_2.a = t1_2.b) + -> Seq Scan on prt1_adv_p2 t3_2 + -> Hash + -> Seq Scan on prt2_adv_p2 t1_2 + Filter: (a = 0) + -> Hash Right Join + Hash Cond: (t2_3.a = t1_3.b) + -> Seq Scan on prt1_adv_p3 t2_3 + -> Hash + -> Hash Join + Hash Cond: (t3_3.a = t1_3.b) + -> Seq Scan on prt1_adv_p3 t3_3 + -> Hash + -> Seq Scan on prt2_adv_p3 t1_3 + Filter: (a = 0) +(31 rows) + +SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a; + b | c | a | c | a | c +-----+------+-----+------+-----+------ + 100 | 0100 | 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 | 125 | 0125 + 200 | 0200 | 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 | 275 | 0275 + 350 | 0350 | 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 | 375 | 0375 +(8 rows) + +DROP TABLE prt2_adv_extra; +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in such cases +ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3; +-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both +CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375); +CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: (t2.b = t1.a) + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3_1 t2_3 + -> Seq Scan on prt2_adv_p3_2 t2_4 + -> Hash + -> Append + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(17 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Semi Join + Hash Cond: (t1.a = t2.b) + -> Append + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3_1 t2_3 + -> Seq Scan on prt2_adv_p3_2 t2_4 +(17 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a, t2.b + -> Hash Right Join + Hash Cond: (t2.b = t1.a) + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3_1 t2_3 + -> Seq Scan on prt2_adv_p3_2 t2_4 + -> Hash + -> Append + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) +(17 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Anti Join + Hash Cond: (t1.a = t2.b) + -> Append + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3_1 t2_3 + -> Seq Scan on prt2_adv_p3_2 t2_4 +(17 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + QUERY PLAN +---------------------------------------------------------------- + Sort + Sort Key: prt1_adv.a, prt2_adv.b + -> Hash Full Join + Hash Cond: (prt2_adv.b = prt1_adv.a) + Filter: (((175) = prt1_adv.a) OR ((425) = prt2_adv.b)) + -> Append + -> Seq Scan on prt2_adv_p1 prt2_adv_1 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p2 prt2_adv_2 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p3_1 prt2_adv_3 + Filter: (a = 0) + -> Seq Scan on prt2_adv_p3_2 prt2_adv_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_adv_p1 prt1_adv_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p2 prt1_adv_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 prt1_adv_3 + Filter: (b = 0) +(22 rows) + +DROP TABLE prt2_adv_p3_1; +DROP TABLE prt2_adv_p3_2; +ANALYZE prt2_adv; +-- Test default partitions +ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1; +-- Change prt1_adv_p1 to the default partition +ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT; +ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3; +ANALYZE prt1_adv; +-- We can do partitioned join even if only one of relations has the default +-- partition +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: (t2_1.b = t1_2.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_2 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_1.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_1 + Filter: (b = 0) +(15 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 +(6 rows) + +-- Restore prt1_adv_p3 +ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400); +ANALYZE prt1_adv; +-- Restore prt2_adv_p3 +ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500); +ANALYZE prt2_adv; +-- Partitioned join can't be applied because the default partition of prt1_adv +-- matches prt2_adv_p1 and prt2_adv_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: (t2.b = t1.a) + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on prt1_adv_p2 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p1 t1_3 + Filter: (b = 0) +(16 rows) + +ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3; +-- Change prt2_adv_p3 to the default partition +ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT; +ANALYZE prt2_adv; +-- Partitioned join can't be applied because the default partition of prt1_adv +-- matches prt2_adv_p1 and prt2_adv_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: (t2.b = t1.a) + -> Append + -> Seq Scan on prt2_adv_p1 t2_1 + -> Seq Scan on prt2_adv_p2 t2_2 + -> Seq Scan on prt2_adv_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on prt1_adv_p2 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p3 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_adv_p1 t1_3 + Filter: (b = 0) +(16 rows) + +DROP TABLE prt1_adv_p3; +ANALYZE prt1_adv; +DROP TABLE prt2_adv_p3; +ANALYZE prt2_adv; +CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt3_adv_a_idx ON prt3_adv (a); +INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i; +ANALYZE prt3_adv; +-- 3-way join to test the default partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a; + QUERY PLAN +------------------------------------------------------------------ + Sort + Sort Key: t1.a, t2.b, t3.a + -> Append + -> Hash Right Join + Hash Cond: (t3_1.a = t1_1.a) + -> Seq Scan on prt3_adv_p1 t3_1 + -> Hash + -> Hash Right Join + Hash Cond: (t2_2.b = t1_1.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t3_2.a = t1_2.a) + -> Seq Scan on prt3_adv_p2 t3_2 + -> Hash + -> Hash Right Join + Hash Cond: (t2_1.b = t1_2.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_2 + Filter: (b = 0) +(23 rows) + +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a; + a | c | b | c | a | c +-----+------+-----+------+-----+------ + 100 | 0100 | 100 | 0100 | | + 125 | 0125 | 125 | 0125 | | + 150 | 0150 | | | | + 175 | 0175 | | | | + 200 | 0200 | 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 | 275 | 0275 +(8 rows) + +DROP TABLE prt1_adv; +DROP TABLE prt2_adv; +DROP TABLE prt3_adv; +-- Test interaction of partitioned join with partition pruning +CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_adv_a_idx ON prt1_adv (a); +INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_adv; +CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400); +CREATE INDEX prt2_adv_b_idx ON prt2_adv (b); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +----------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: ((a < 300) AND (b = 0)) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: ((a < 300) AND (b = 0)) +(15 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 150 | 0150 | 150 | 0150 + 175 | 0175 | 175 | 0175 + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 +(8 rows) + +DROP TABLE prt1_adv_p3; +CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT; +ANALYZE prt1_adv; +CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT; +ANALYZE prt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + QUERY PLAN +-------------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on prt1_adv_p1 t1_1 + Filter: ((a >= 100) AND (a < 300) AND (b = 0)) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on prt1_adv_p2 t1_2 + Filter: ((a >= 100) AND (a < 300) AND (b = 0)) +(15 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + a | c | b | c +-----+------+-----+------ + 100 | 0100 | 100 | 0100 + 125 | 0125 | 125 | 0125 + 150 | 0150 | 150 | 0150 + 175 | 0175 | 175 | 0175 + 200 | 0200 | 200 | 0200 + 225 | 0225 | 225 | 0225 + 250 | 0250 | 250 | 0250 + 275 | 0275 | 275 | 0275 +(8 rows) + +DROP TABLE prt1_adv; +DROP TABLE prt2_adv; +-- Tests for list-partitioned tables +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +ANALYZE plt1_adv; +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 9 | 0009 | 9 | 0009 +(4 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Nested Loop Semi Join + Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Semi Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Semi Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +---+---+------ + 3 | 3 | 0003 + 4 | 4 | 0004 + 6 | 6 | 0006 + 9 | 9 | 0009 +(4 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 +(6 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Nested Loop Anti Join + Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Anti Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Anti Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +---+---+------ + 1 | 1 | 0001 + 8 | 8 | 0008 +(2 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +----------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Append + -> Hash Full Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10)) + -> Seq Scan on plt1_adv_p1 t1_1 + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash Full Join + Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10)) + -> Seq Scan on plt1_adv_p2 t1_2 + -> Hash + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash Full Join + Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10)) + -> Seq Scan on plt1_adv_p3 t1_3 + -> Hash + -> Seq Scan on plt2_adv_p3 t2_3 +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + a | c | a | c +---+------+---+------ + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 + | | 2 | 0002 + | | 7 | 0007 +(8 rows) + +-- Test cases where one side has an extra partition +CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000'); +INSERT INTO plt2_adv_extra VALUES (0, 0, '0000'); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 9 | 0009 | 9 | 0009 +(4 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Nested Loop Semi Join + Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Semi Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Semi Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +---+---+------ + 3 | 3 | 0003 + 4 | 4 | 0004 + 6 | 6 | 0006 + 9 | 9 | 0009 +(4 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 +(6 rows) + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +--------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Right Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt1_adv_p1 t2_1 + -> Seq Scan on plt1_adv_p2 t2_2 + -> Seq Scan on plt1_adv_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on plt2_adv_extra t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t1_4 + Filter: (b < 10) +(18 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Nested Loop Anti Join + Join Filter: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Anti Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Anti Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +---+---+------ + 1 | 1 | 0001 + 8 | 8 | 0008 +(2 rows) + +-- anti join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Anti Join + Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c)) + -> Append + -> Seq Scan on plt2_adv_extra t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p1 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3 t1_4 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t2_1 + -> Seq Scan on plt1_adv_p2 t2_2 + -> Seq Scan on plt1_adv_p3 t2_3 +(18 rows) + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Hash Full Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10)) + -> Append + -> Seq Scan on plt2_adv_extra t2_1 + -> Seq Scan on plt2_adv_p1 t2_2 + -> Seq Scan on plt2_adv_p2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + -> Seq Scan on plt1_adv_p2 t1_2 + -> Seq Scan on plt1_adv_p3 t1_3 +(15 rows) + +DROP TABLE plt2_adv_extra; +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in such cases +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2; +-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both +CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004'); +CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_1 t2_2 + -> Seq Scan on plt2_adv_p2_2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(17 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Semi Join + Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c)) + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_1 t2_2 + -> Seq Scan on plt2_adv_p2_2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 +(17 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Right Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_1 t2_2 + -> Seq Scan on plt2_adv_p2_2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(17 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Anti Join + Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c)) + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_1 t2_2 + -> Seq Scan on plt2_adv_p2_2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 +(17 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Hash Full Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_1 t2_2 + -> Seq Scan on plt2_adv_p2_2 t2_3 + -> Seq Scan on plt2_adv_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + -> Seq Scan on plt1_adv_p2 t1_2 + -> Seq Scan on plt1_adv_p3 t1_3 +(15 rows) + +DROP TABLE plt2_adv_p2_1; +DROP TABLE plt2_adv_p2_2; +-- Restore plt2_adv_p2 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006'); +-- Test NULL partitions +ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1; +-- Change plt1_adv_p1 to the NULL partition +CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3; +-- Change plt2_adv_p3 to the NULL partition +CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1_null t1_1 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 9 | 0009 | 9 | 0009 +(4 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Semi Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1_null t1_1 + Filter: (b < 10) + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Semi Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Semi Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3_null t2_3 +(19 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +---+---+------ + 3 | 3 | 0003 + 4 | 4 | 0004 + 6 | 6 | 0006 + 9 | 9 | 0009 +(4 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1_null t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +----+------+---+------ + -1 | | | + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 +(7 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Anti Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + -> Seq Scan on plt1_adv_p1_null t1_1 + Filter: (b < 10) + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Nested Loop Anti Join + Join Filter: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Nested Loop Anti Join + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_adv_p3_null t2_3 +(19 rows) + +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + a | b | c +----+----+------ + -1 | -1 | + 1 | 1 | 0001 + 8 | 8 | 0008 +(3 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +----------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Append + -> Hash Full Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10)) + -> Seq Scan on plt1_adv_p1_null t1_1 + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash Full Join + Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10)) + -> Seq Scan on plt1_adv_p2 t1_2 + -> Hash + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash Full Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10)) + -> Seq Scan on plt2_adv_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + a | c | a | c +----+------+----+------ + -1 | | | + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 + | | -1 | + | | 2 | 0002 + | | 7 | 0007 +(10 rows) + +DROP TABLE plt1_adv_p1_null; +-- Restore plt1_adv_p1 +ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003'); +-- Add to plt1_adv the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; +DROP TABLE plt2_adv_p3_null; +-- Restore plt2_adv_p3 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009'); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 9 | 0009 | 9 | 0009 +(4 rows) + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +--------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Right Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2 t2_2 + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt1_adv_extra t1_4 + Filter: (b < 10) +(18 rows) + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Hash Full Join + Hash Cond: ((t1.a = t2.a) AND (t1.c = t2.c)) + Filter: ((COALESCE(t1.b, 0) < 10) AND (COALESCE(t2.b, 0) < 10)) + -> Append + -> Seq Scan on plt1_adv_p1 t1_1 + -> Seq Scan on plt1_adv_p2 t1_2 + -> Seq Scan on plt1_adv_p3 t1_3 + -> Seq Scan on plt1_adv_extra t1_4 + -> Hash + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2 t2_2 + -> Seq Scan on plt2_adv_p3 t2_3 +(15 rows) + +-- Add to plt2_adv the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 9 | 0009 | 9 | 0009 +(4 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +---------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Nested Loop Left Join + Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c)) + -> Seq Scan on plt1_adv_extra t1_4 + Filter: (b < 10) + -> Seq Scan on plt2_adv_extra t2_4 +(26 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +----+------+---+------ + -1 | | | + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 +(7 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + QUERY PLAN +----------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t2.a + -> Append + -> Hash Full Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.c = t2_1.c)) + Filter: ((COALESCE(t1_1.b, 0) < 10) AND (COALESCE(t2_1.b, 0) < 10)) + -> Seq Scan on plt1_adv_p1 t1_1 + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash Full Join + Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + Filter: ((COALESCE(t1_2.b, 0) < 10) AND (COALESCE(t2_2.b, 0) < 10)) + -> Seq Scan on plt1_adv_p2 t1_2 + -> Hash + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash Full Join + Hash Cond: ((t1_3.a = t2_3.a) AND (t1_3.c = t2_3.c)) + Filter: ((COALESCE(t1_3.b, 0) < 10) AND (COALESCE(t2_3.b, 0) < 10)) + -> Seq Scan on plt1_adv_p3 t1_3 + -> Hash + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash Full Join + Hash Cond: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c)) + Filter: ((COALESCE(t1_4.b, 0) < 10) AND (COALESCE(t2_4.b, 0) < 10)) + -> Seq Scan on plt1_adv_extra t1_4 + -> Hash + -> Seq Scan on plt2_adv_extra t2_4 +(27 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + a | c | a | c +----+------+----+------ + -1 | | | + 1 | 0001 | | + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 + 8 | 0008 | | + 9 | 0009 | 9 | 0009 + | | -1 | + | | 2 | 0002 + | | 7 | 0007 +(10 rows) + +-- 3-way join to test the NULL partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c)) + -> Seq Scan on plt1_adv_p1 t3_1 + -> Hash + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c)) + -> Seq Scan on plt1_adv_p2 t3_2 + -> Hash + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_2 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t3_3.a = t1_3.a) AND (t3_3.c = t1_3.c)) + -> Seq Scan on plt1_adv_p3 t3_3 + -> Hash + -> Hash Right Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.c = t1_3.c)) + -> Seq Scan on plt2_adv_p3 t2_3 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_3 + Filter: (b < 10) + -> Nested Loop Left Join + Join Filter: ((t1_4.a = t3_4.a) AND (t1_4.c = t3_4.c)) + -> Nested Loop Left Join + Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c)) + -> Seq Scan on plt1_adv_extra t1_4 + Filter: (b < 10) + -> Seq Scan on plt2_adv_extra t2_4 + -> Seq Scan on plt1_adv_extra t3_4 +(41 rows) + +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c | a | c +----+------+---+------+---+------ + -1 | | | | | + 1 | 0001 | | | 1 | 0001 + 3 | 0003 | 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 | 6 | 0006 + 8 | 0008 | | | 8 | 0008 + 9 | 0009 | 9 | 0009 | 9 | 0009 +(7 rows) + +DROP TABLE plt1_adv_extra; +DROP TABLE plt2_adv_extra; +-- Test default partitions +ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1; +-- Change plt1_adv_p1 to the default partition +ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT; +DROP TABLE plt1_adv_p3; +ANALYZE plt1_adv; +DROP TABLE plt2_adv_p3; +ANALYZE plt2_adv; +-- We can do partitioned join even if only one of relations has the default +-- partition +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_1 + Filter: (b < 10) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 +(3 rows) + +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2; +-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as +-- the key values +CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6); +ANALYZE plt2_adv; +-- Partitioned join can't be applied because the default partition of plt1_adv +-- matches plt2_adv_p1 and plt2_adv_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_ext t2_2 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p2 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p1 t1_2 + Filter: (b < 10) +(13 rows) + +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext; +-- Change plt2_adv_p2_ext to the default partition +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT; +ANALYZE plt2_adv; +-- Partitioned join can't be applied because the default partition of plt1_adv +-- matches plt2_adv_p1 and plt2_adv_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Append + -> Seq Scan on plt2_adv_p1 t2_1 + -> Seq Scan on plt2_adv_p2_ext t2_2 + -> Hash + -> Append + -> Seq Scan on plt1_adv_p2 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_adv_p1 t1_2 + Filter: (b < 10) +(13 rows) + +DROP TABLE plt2_adv_p2_ext; +-- Restore plt2_adv_p2 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_adv; +CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009'); +INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9); +ANALYZE plt3_adv; +-- 3-way join to test the default partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Right Join + Hash Cond: ((t3_1.a = t1_1.a) AND (t3_1.c = t1_1.c)) + -> Seq Scan on plt3_adv_p1 t3_1 + -> Hash + -> Hash Right Join + Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_1 + Filter: (b < 10) + -> Hash Right Join + Hash Cond: ((t3_2.a = t1_2.a) AND (t3_2.c = t1_2.c)) + -> Seq Scan on plt3_adv_p2 t3_2 + -> Hash + -> Hash Right Join + Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c)) + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_2 + Filter: (b < 10) +(23 rows) + +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c | a | c +---+------+---+------+---+------ + 1 | 0001 | | | | + 3 | 0003 | 3 | 0003 | | + 4 | 0004 | 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 | 6 | 0006 +(4 rows) + +-- Test cases where one side has the default partition while the other side +-- has the NULL partition +DROP TABLE plt2_adv_p1; +-- Add the NULL partition to plt2_adv +CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_2.a) AND (t2_1.c = t1_2.c)) + -> Seq Scan on plt2_adv_p1_null t2_1 + -> Hash + -> Seq Scan on plt1_adv_p1 t1_2 + Filter: (b < 10) + -> Hash Join + Hash Cond: ((t2_2.a = t1_1.a) AND (t2_2.c = t1_1.c)) + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1_1 + Filter: (b < 10) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 1 | 0001 | 1 | 0001 + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 +(4 rows) + +DROP TABLE plt2_adv_p1_null; +-- Add the NULL partition that contains only NULL values as the key values +CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + QUERY PLAN +------------------------------------------------------ + Sort + Sort Key: t1.a + -> Hash Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Seq Scan on plt2_adv_p2 t2 + -> Hash + -> Seq Scan on plt1_adv_p2 t1 + Filter: (b < 10) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 4 | 0004 | 4 | 0004 + 6 | 0006 | 6 | 0006 +(2 rows) + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; +DROP TABLE plt3_adv; +-- Test interaction of partitioned join with partition pruning +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002'); +CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003'); +CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL); +CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003'); +CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +----------------------------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p3 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_1 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p4 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p4 t1_2 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 5 | 0005 | 5 | 0005 +(3 rows) + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Right Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Seq Scan on plt2_adv_p4 t2 + -> Hash + -> Seq Scan on plt1_adv_p4 t1 + Filter: ((c IS NULL) AND (b < 10)) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + a | c | a | c +----+---+---+--- + -1 | | | +(1 row) + +CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT; +ANALYZE plt1_adv; +CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT; +ANALYZE plt2_adv; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +----------------------------------------------------------------------------------------- + Sort + Sort Key: t1.a + -> Append + -> Hash Join + Hash Cond: ((t2_1.a = t1_1.a) AND (t2_1.c = t1_1.c)) + -> Seq Scan on plt2_adv_p3 t2_1 + -> Hash + -> Seq Scan on plt1_adv_p3 t1_1 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.c = t1_2.c)) + -> Seq Scan on plt2_adv_p4 t2_2 + -> Hash + -> Seq Scan on plt1_adv_p4 t1_2 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + a | c | a | c +---+------+---+------ + 3 | 0003 | 3 | 0003 + 4 | 0004 | 4 | 0004 + 5 | 0005 | 5 | 0005 +(3 rows) + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + QUERY PLAN +-------------------------------------------------------- + Sort + Sort Key: t1.a + -> Hash Right Join + Hash Cond: ((t2.a = t1.a) AND (t2.c = t1.c)) + -> Seq Scan on plt2_adv_p4 t2 + -> Hash + -> Seq Scan on plt1_adv_p4 t1 + Filter: ((c IS NULL) AND (b < 10)) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + a | c | a | c +----+---+---+--- + -1 | | | +(1 row) + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; +-- Test the process_outer_partition() code path +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i; +ANALYZE plt1_adv; +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4); +ANALYZE plt2_adv; +CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001'); +CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4); +ANALYZE plt3_adv; +-- This tests that when merging partitions from plt1_adv and plt2_adv in +-- merge_list_bounds(), process_outer_partition() returns an already-assigned +-- merged partition when re-called with plt1_adv_p1 for the second list value +-- '0001' of that partitin +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a; + QUERY PLAN +----------------------------------------------------------------------------------------------- + Sort + Sort Key: t1.c, t1.a, t2.a, t3.a + -> Append + -> Hash Full Join + Hash Cond: (t1_1.c = t3_1.c) + Filter: (((COALESCE(t1_1.a, 0) % 5) <> 3) AND ((COALESCE(t1_1.a, 0) % 5) <> 4)) + -> Hash Left Join + Hash Cond: (t1_1.c = t2_1.c) + -> Seq Scan on plt1_adv_p1 t1_1 + -> Hash + -> Seq Scan on plt2_adv_p1 t2_1 + -> Hash + -> Seq Scan on plt3_adv_p1 t3_1 + -> Hash Full Join + Hash Cond: (t1_2.c = t3_2.c) + Filter: (((COALESCE(t1_2.a, 0) % 5) <> 3) AND ((COALESCE(t1_2.a, 0) % 5) <> 4)) + -> Hash Left Join + Hash Cond: (t1_2.c = t2_2.c) + -> Seq Scan on plt1_adv_p2 t1_2 + -> Hash + -> Seq Scan on plt2_adv_p2 t2_2 + -> Hash + -> Seq Scan on plt3_adv_p2 t3_2 +(23 rows) + +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a; + a | c | a | c | a | c +----+------+----+------+----+------ + 0 | 0000 | | | | + 5 | 0000 | | | | + 10 | 0000 | | | | + 15 | 0000 | | | | + 20 | 0000 | | | | + 1 | 0001 | | | 1 | 0001 + 1 | 0001 | | | 6 | 0001 + 1 | 0001 | | | 11 | 0001 + 1 | 0001 | | | 16 | 0001 + 1 | 0001 | | | 21 | 0001 + 6 | 0001 | | | 1 | 0001 + 6 | 0001 | | | 6 | 0001 + 6 | 0001 | | | 11 | 0001 + 6 | 0001 | | | 16 | 0001 + 6 | 0001 | | | 21 | 0001 + 11 | 0001 | | | 1 | 0001 + 11 | 0001 | | | 6 | 0001 + 11 | 0001 | | | 11 | 0001 + 11 | 0001 | | | 16 | 0001 + 11 | 0001 | | | 21 | 0001 + 16 | 0001 | | | 1 | 0001 + 16 | 0001 | | | 6 | 0001 + 16 | 0001 | | | 11 | 0001 + 16 | 0001 | | | 16 | 0001 + 16 | 0001 | | | 21 | 0001 + 21 | 0001 | | | 1 | 0001 + 21 | 0001 | | | 6 | 0001 + 21 | 0001 | | | 11 | 0001 + 21 | 0001 | | | 16 | 0001 + 21 | 0001 | | | 21 | 0001 + 2 | 0002 | 2 | 0002 | | + 2 | 0002 | 7 | 0002 | | + 2 | 0002 | 12 | 0002 | | + 2 | 0002 | 17 | 0002 | | + 2 | 0002 | 22 | 0002 | | + 7 | 0002 | 2 | 0002 | | + 7 | 0002 | 7 | 0002 | | + 7 | 0002 | 12 | 0002 | | + 7 | 0002 | 17 | 0002 | | + 7 | 0002 | 22 | 0002 | | + 12 | 0002 | 2 | 0002 | | + 12 | 0002 | 7 | 0002 | | + 12 | 0002 | 12 | 0002 | | + 12 | 0002 | 17 | 0002 | | + 12 | 0002 | 22 | 0002 | | + 17 | 0002 | 2 | 0002 | | + 17 | 0002 | 7 | 0002 | | + 17 | 0002 | 12 | 0002 | | + 17 | 0002 | 17 | 0002 | | + 17 | 0002 | 22 | 0002 | | + 22 | 0002 | 2 | 0002 | | + 22 | 0002 | 7 | 0002 | | + 22 | 0002 | 12 | 0002 | | + 22 | 0002 | 17 | 0002 | | + 22 | 0002 | 22 | 0002 | | +(55 rows) + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; +DROP TABLE plt3_adv; +-- Tests for multi-level partitioned tables +CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a); +CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b); +CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c); +CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200); +CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300); +CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400); +CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003'); +CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006'); +CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009'); +INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +INSERT INTO alpha_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +ANALYZE alpha; +CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a); +CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b); +CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c); +CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150); +CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300); +CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500); +CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003'); +CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006'); +CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009'); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +ANALYZE beta; +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b; + QUERY PLAN +-------------------------------------------------------------------- + Sort + Sort Key: t1.a, t1.b + -> Append + -> Hash Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b)) + -> Seq Scan on alpha_neg_p1 t1_1 + Filter: ((b >= 125) AND (b < 225)) + -> Hash + -> Seq Scan on beta_neg_p1 t2_1 + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b)) + -> Seq Scan on beta_neg_p2 t2_2 + -> Hash + -> Seq Scan on alpha_neg_p2 t1_2 + Filter: ((b >= 125) AND (b < 225)) + -> Hash Join + Hash Cond: ((t2_4.a = t1_4.a) AND (t2_4.b = t1_4.b)) + -> Append + -> Seq Scan on beta_pos_p1 t2_4 + -> Seq Scan on beta_pos_p2 t2_5 + -> Seq Scan on beta_pos_p3 t2_6 + -> Hash + -> Append + -> Seq Scan on alpha_pos_p1 t1_4 + Filter: ((b >= 125) AND (b < 225)) + -> Seq Scan on alpha_pos_p2 t1_5 + Filter: ((b >= 125) AND (b < 225)) + -> Seq Scan on alpha_pos_p3 t1_6 + Filter: ((b >= 125) AND (b < 225)) +(29 rows) + +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b; + a | b | c | a | b | c +----+-----+------+----+-----+------ + -1 | 126 | 0006 | -1 | 126 | 0006 + -1 | 129 | 0009 | -1 | 129 | 0009 + -1 | 133 | 0003 | -1 | 133 | 0003 + -1 | 134 | 0004 | -1 | 134 | 0004 + -1 | 136 | 0006 | -1 | 136 | 0006 + -1 | 139 | 0009 | -1 | 139 | 0009 + -1 | 143 | 0003 | -1 | 143 | 0003 + -1 | 144 | 0004 | -1 | 144 | 0004 + -1 | 146 | 0006 | -1 | 146 | 0006 + -1 | 149 | 0009 | -1 | 149 | 0009 + -1 | 203 | 0003 | -1 | 203 | 0003 + -1 | 204 | 0004 | -1 | 204 | 0004 + -1 | 206 | 0006 | -1 | 206 | 0006 + -1 | 209 | 0009 | -1 | 209 | 0009 + -1 | 213 | 0003 | -1 | 213 | 0003 + -1 | 214 | 0004 | -1 | 214 | 0004 + -1 | 216 | 0006 | -1 | 216 | 0006 + -1 | 219 | 0009 | -1 | 219 | 0009 + -1 | 223 | 0003 | -1 | 223 | 0003 + -1 | 224 | 0004 | -1 | 224 | 0004 + 1 | 126 | 0006 | 1 | 126 | 0006 + 1 | 129 | 0009 | 1 | 129 | 0009 + 1 | 133 | 0003 | 1 | 133 | 0003 + 1 | 134 | 0004 | 1 | 134 | 0004 + 1 | 136 | 0006 | 1 | 136 | 0006 + 1 | 139 | 0009 | 1 | 139 | 0009 + 1 | 143 | 0003 | 1 | 143 | 0003 + 1 | 144 | 0004 | 1 | 144 | 0004 + 1 | 146 | 0006 | 1 | 146 | 0006 + 1 | 149 | 0009 | 1 | 149 | 0009 + 1 | 203 | 0003 | 1 | 203 | 0003 + 1 | 204 | 0004 | 1 | 204 | 0004 + 1 | 206 | 0006 | 1 | 206 | 0006 + 1 | 209 | 0009 | 1 | 209 | 0009 + 1 | 213 | 0003 | 1 | 213 | 0003 + 1 | 214 | 0004 | 1 | 214 | 0004 + 1 | 216 | 0006 | 1 | 216 | 0006 + 1 | 219 | 0009 | 1 | 219 | 0009 + 1 | 223 | 0003 | 1 | 223 | 0003 + 1 | 224 | 0004 | 1 | 224 | 0004 +(40 rows) + +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b; + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t1.b, t2.b + -> Append + -> Hash Join + Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.c = t2_2.c)) + -> Append + -> Seq Scan on alpha_neg_p1 t1_2 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Seq Scan on alpha_neg_p2 t1_3 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Hash + -> Append + -> Seq Scan on beta_neg_p1 t2_2 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Seq Scan on beta_neg_p2 t2_3 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Nested Loop + Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.c = t2_4.c)) + -> Seq Scan on alpha_pos_p2 t1_4 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Seq Scan on beta_pos_p2 t2_4 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Nested Loop + Join Filter: ((t1_5.a = t2_5.a) AND (t1_5.c = t2_5.c)) + -> Seq Scan on alpha_pos_p3 t1_5 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Seq Scan on beta_pos_p3 t2_5 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) +(28 rows) + +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b; + a | b | c | a | b | c +----+-----+------+----+-----+------ + -1 | 104 | 0004 | -1 | 104 | 0004 + -1 | 104 | 0004 | -1 | 204 | 0004 + -1 | 109 | 0009 | -1 | 109 | 0009 + -1 | 109 | 0009 | -1 | 209 | 0009 + -1 | 204 | 0004 | -1 | 104 | 0004 + -1 | 204 | 0004 | -1 | 204 | 0004 + -1 | 209 | 0009 | -1 | 109 | 0009 + -1 | 209 | 0009 | -1 | 209 | 0009 + 1 | 104 | 0004 | 1 | 104 | 0004 + 1 | 104 | 0004 | 1 | 204 | 0004 + 1 | 109 | 0009 | 1 | 109 | 0009 + 1 | 109 | 0009 | 1 | 209 | 0009 + 1 | 204 | 0004 | 1 | 104 | 0004 + 1 | 204 | 0004 | 1 | 204 | 0004 + 1 | 209 | 0009 | 1 | 109 | 0009 + 1 | 209 | 0009 | 1 | 209 | 0009 (16 rows) +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b; + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t1.b + -> Append + -> Hash Join + Hash Cond: ((t1_1.a = t2_1.a) AND (t1_1.b = t2_1.b) AND (t1_1.c = t2_1.c)) + -> Seq Scan on alpha_neg_p1 t1_1 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Hash + -> Seq Scan on beta_neg_p1 t2_1 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Hash Join + Hash Cond: ((t1_2.a = t2_2.a) AND (t1_2.b = t2_2.b) AND (t1_2.c = t2_2.c)) + -> Seq Scan on alpha_neg_p2 t1_2 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Hash + -> Seq Scan on beta_neg_p2 t2_2 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Nested Loop + Join Filter: ((t1_3.a = t2_3.a) AND (t1_3.b = t2_3.b) AND (t1_3.c = t2_3.c)) + -> Seq Scan on alpha_pos_p2 t1_3 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Seq Scan on beta_pos_p2 t2_3 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) + -> Nested Loop + Join Filter: ((t1_4.a = t2_4.a) AND (t1_4.b = t2_4.b) AND (t1_4.c = t2_4.c)) + -> Seq Scan on alpha_pos_p3 t1_4 + Filter: ((c = ANY ('{0004,0009}'::text[])) AND (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210)))) + -> Seq Scan on beta_pos_p3 t2_4 + Filter: (((b >= 100) AND (b < 110)) OR ((b >= 200) AND (b < 210))) +(29 rows) + +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b; + a | b | c | a | b | c +----+-----+------+----+-----+------ + -1 | 104 | 0004 | -1 | 104 | 0004 + -1 | 109 | 0009 | -1 | 109 | 0009 + -1 | 204 | 0004 | -1 | 204 | 0004 + -1 | 209 | 0009 | -1 | 209 | 0009 + 1 | 104 | 0004 | 1 | 104 | 0004 + 1 | 109 | 0009 | 1 | 109 | 0009 + 1 | 204 | 0004 | 1 | 204 | 0004 + 1 | 209 | 0009 | 1 | 209 | 0009 +(8 rows) + diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql index 575ba7b8d4..df30f851e8 100644 --- a/src/test/regress/sql/partition_join.sql +++ b/src/test/regress/sql/partition_join.sql @@ -445,11 +445,638 @@ SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 JOIN prt2_n t2 ON (t1.c = t2.c) JOI EXPLAIN (COSTS OFF) SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_n t1 FULL JOIN prt1 t2 ON (t1.c = t2.c); --- partitionwise join can not be applied if only one of joining tables has --- default partition -ALTER TABLE prt2 DETACH PARTITION prt2_p3; -ALTER TABLE prt2 ATTACH PARTITION prt2_p3 FOR VALUES FROM (500) TO (600); -ANALYZE prt2; +-- +-- Test advanced partition-matching algorithm for partitioned join +-- + +-- Tests for range-partitioned tables +CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_adv_a_idx ON prt1_adv (a); +INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_adv; + +CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (150); +CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt2_adv_p3 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (500); +CREATE INDEX prt2_adv_b_idx ON prt2_adv (b); +INSERT INTO prt2_adv_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i; +INSERT INTO prt2_adv_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i; +INSERT INTO prt2_adv_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_adv; + +-- inner join EXPLAIN (COSTS OFF) -SELECT t1.a, t1.c, t2.b, t2.c FROM prt1 t1, prt2 t2 WHERE t1.a = t2.b AND t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + +-- Test cases where one side has an extra partition +CREATE TABLE prt2_adv_extra PARTITION OF prt2_adv FOR VALUES FROM (500) TO (MAXVALUE); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i; +ANALYZE prt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.b, t1.c, t2.a, t2.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- anti join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_adv t2 WHERE t1.b = t2.a) AND t1.a = 0 ORDER BY t1.b; + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + +-- 3-way join where not every pair of relations can do partitioned join +EXPLAIN (COSTS OFF) +SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a; +SELECT t1.b, t1.c, t2.a, t2.c, t3.a, t3.c FROM prt2_adv t1 LEFT JOIN prt1_adv t2 ON (t1.b = t2.a) INNER JOIN prt1_adv t3 ON (t1.b = t3.a) WHERE t1.a = 0 ORDER BY t1.b, t2.a, t3.a; + +DROP TABLE prt2_adv_extra; + +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in such cases +ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3; +-- Split prt2_adv_p3 into two partitions so that prt1_adv_p3 matches both +CREATE TABLE prt2_adv_p3_1 PARTITION OF prt2_adv FOR VALUES FROM (350) TO (375); +CREATE TABLE prt2_adv_p3_2 PARTITION OF prt2_adv FOR VALUES FROM (375) TO (500); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_adv t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_adv WHERE prt1_adv.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_adv WHERE prt2_adv.a = 0) t2 ON (t1.a = t2.b) WHERE t1.phv = t1.a OR t2.phv = t2.b ORDER BY t1.a, t2.b; + +DROP TABLE prt2_adv_p3_1; +DROP TABLE prt2_adv_p3_2; +ANALYZE prt2_adv; + +-- Test default partitions +ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p1; +-- Change prt1_adv_p1 to the default partition +ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p1 DEFAULT; +ALTER TABLE prt1_adv DETACH PARTITION prt1_adv_p3; +ANALYZE prt1_adv; + +-- We can do partitioned join even if only one of relations has the default +-- partition +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- Restore prt1_adv_p3 +ALTER TABLE prt1_adv ATTACH PARTITION prt1_adv_p3 FOR VALUES FROM (300) TO (400); +ANALYZE prt1_adv; + +-- Restore prt2_adv_p3 +ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 FOR VALUES FROM (350) TO (500); +ANALYZE prt2_adv; + +-- Partitioned join can't be applied because the default partition of prt1_adv +-- matches prt2_adv_p1 and prt2_adv_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +ALTER TABLE prt2_adv DETACH PARTITION prt2_adv_p3; +-- Change prt2_adv_p3 to the default partition +ALTER TABLE prt2_adv ATTACH PARTITION prt2_adv_p3 DEFAULT; +ANALYZE prt2_adv; + +-- Partitioned join can't be applied because the default partition of prt1_adv +-- matches prt2_adv_p1 and prt2_adv_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt1_adv_p3; +ANALYZE prt1_adv; + +DROP TABLE prt2_adv_p3; +ANALYZE prt2_adv; + +CREATE TABLE prt3_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt3_adv_p1 PARTITION OF prt3_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt3_adv_p2 PARTITION OF prt3_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt3_adv_a_idx ON prt3_adv (a); +INSERT INTO prt3_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i; +ANALYZE prt3_adv; + +-- 3-way join to test the default partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a; +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_adv t1 LEFT JOIN prt2_adv t2 ON (t1.a = t2.b) LEFT JOIN prt3_adv t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a; + +DROP TABLE prt1_adv; +DROP TABLE prt2_adv; +DROP TABLE prt3_adv; + +-- Test interaction of partitioned join with partition pruning +CREATE TABLE prt1_adv (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_adv_p1 PARTITION OF prt1_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_adv_p2 PARTITION OF prt1_adv FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_adv_p3 PARTITION OF prt1_adv FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_adv_a_idx ON prt1_adv (a); +INSERT INTO prt1_adv SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_adv; + +CREATE TABLE prt2_adv (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_adv_p1 PARTITION OF prt2_adv FOR VALUES FROM (100) TO (200); +CREATE TABLE prt2_adv_p2 PARTITION OF prt2_adv FOR VALUES FROM (200) TO (400); +CREATE INDEX prt2_adv_b_idx ON prt2_adv (b); +INSERT INTO prt2_adv SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt1_adv_p3; +CREATE TABLE prt1_adv_default PARTITION OF prt1_adv DEFAULT; +ANALYZE prt1_adv; + +CREATE TABLE prt2_adv_default PARTITION OF prt2_adv DEFAULT; +ANALYZE prt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_adv t1 INNER JOIN prt2_adv t2 ON (t1.a = t2.b) WHERE t1.a >= 100 AND t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt1_adv; +DROP TABLE prt2_adv; + + +-- Tests for list-partitioned tables +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001', '0003'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0008', '0009'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +ANALYZE plt1_adv; + +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002', '0003'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0007', '0009'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +-- Test cases where one side has an extra partition +CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN ('0000'); +INSERT INTO plt2_adv_extra VALUES (0, 0, '0000'); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt2_adv t1 LEFT JOIN plt1_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- anti join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt2_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +DROP TABLE plt2_adv_extra; + +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in such cases +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2; +-- Split plt2_adv_p2 into two partitions so that plt1_adv_p2 matches both +CREATE TABLE plt2_adv_p2_1 PARTITION OF plt2_adv FOR VALUES IN ('0004'); +CREATE TABLE plt2_adv_p2_2 PARTITION OF plt2_adv FOR VALUES IN ('0006'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +DROP TABLE plt2_adv_p2_1; +DROP TABLE plt2_adv_p2_2; +-- Restore plt2_adv_p2 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006'); + +-- Test NULL partitions +ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1; +-- Change plt1_adv_p1 to the NULL partition +CREATE TABLE plt1_adv_p1_null PARTITION OF plt1_adv FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; + +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p3; +-- Change plt2_adv_p3 to the NULL partition +CREATE TABLE plt2_adv_p3_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0007', '0009'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_adv t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_adv t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +DROP TABLE plt1_adv_p1_null; +-- Restore plt1_adv_p1 +ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 FOR VALUES IN ('0001', '0003'); +-- Add to plt1_adv the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt1_adv_extra PARTITION OF plt1_adv FOR VALUES IN (NULL); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; + +DROP TABLE plt2_adv_p3_null; +-- Restore plt2_adv_p3 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p3 FOR VALUES IN ('0007', '0009'); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- left join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- full join; currently we can't do partitioned join if there are no matched +-- partitions on the nullable side +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +-- Add to plt2_adv the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt2_adv_extra PARTITION OF plt2_adv FOR VALUES IN (NULL); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 FULL JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE coalesce(t1.b, 0) < 10 AND coalesce(t2.b, 0) < 10 ORDER BY t1.a, t2.a; + +-- 3-way join to test the NULL partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt1_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt1_adv_extra; +DROP TABLE plt2_adv_extra; + +-- Test default partitions +ALTER TABLE plt1_adv DETACH PARTITION plt1_adv_p1; +-- Change plt1_adv_p1 to the default partition +ALTER TABLE plt1_adv ATTACH PARTITION plt1_adv_p1 DEFAULT; +DROP TABLE plt1_adv_p3; +ANALYZE plt1_adv; + +DROP TABLE plt2_adv_p3; +ANALYZE plt2_adv; + +-- We can do partitioned join even if only one of relations has the default +-- partition +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2; +-- Change plt2_adv_p2 to contain '0005' in addition to '0004' and '0006' as +-- the key values +CREATE TABLE plt2_adv_p2_ext PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005', '0006'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6); +ANALYZE plt2_adv; + +-- Partitioned join can't be applied because the default partition of plt1_adv +-- matches plt2_adv_p1 and plt2_adv_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +ALTER TABLE plt2_adv DETACH PARTITION plt2_adv_p2_ext; +-- Change plt2_adv_p2_ext to the default partition +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2_ext DEFAULT; +ANALYZE plt2_adv; + +-- Partitioned join can't be applied because the default partition of plt1_adv +-- matches plt2_adv_p1 and plt2_adv_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt2_adv_p2_ext; +-- Restore plt2_adv_p2 +ALTER TABLE plt2_adv ATTACH PARTITION plt2_adv_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_adv; + +CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0007', '0009'); +INSERT INTO plt3_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9); +ANALYZE plt3_adv; + +-- 3-way join to test the default partition of a join relation +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_adv t3 ON (t1.a = t3.a AND t1.c = t3.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Test cases where one side has the default partition while the other side +-- has the NULL partition +DROP TABLE plt2_adv_p1; +-- Add the NULL partition to plt2_adv +CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt2_adv_p1_null; +-- Add the NULL partition that contains only NULL values as the key values +CREATE TABLE plt2_adv_p1_null PARTITION OF plt2_adv FOR VALUES IN (NULL); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; +DROP TABLE plt3_adv; + +-- Test interaction of partitioned join with partition pruning +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0001'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0002'); +CREATE TABLE plt1_adv_p3 PARTITION OF plt1_adv FOR VALUES IN ('0003'); +CREATE TABLE plt1_adv_p4 PARTITION OF plt1_adv FOR VALUES IN (NULL, '0004', '0005'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5); +INSERT INTO plt1_adv VALUES (-1, -1, NULL); +ANALYZE plt1_adv; + +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0001', '0002'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN (NULL); +CREATE TABLE plt2_adv_p3 PARTITION OF plt2_adv FOR VALUES IN ('0003'); +CREATE TABLE plt2_adv_p4 PARTITION OF plt2_adv FOR VALUES IN ('0004', '0005'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 2, 3, 4, 5); +INSERT INTO plt2_adv VALUES (-1, -1, NULL); +ANALYZE plt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + +CREATE TABLE plt1_adv_default PARTITION OF plt1_adv DEFAULT; +ANALYZE plt1_adv; + +CREATE TABLE plt2_adv_default PARTITION OF plt2_adv DEFAULT; +ANALYZE plt2_adv; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 INNER JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IS NULL AND t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; + +-- Test the process_outer_partition() code path +CREATE TABLE plt1_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_adv_p1 PARTITION OF plt1_adv FOR VALUES IN ('0000', '0001', '0002'); +CREATE TABLE plt1_adv_p2 PARTITION OF plt1_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt1_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i; +ANALYZE plt1_adv; + +CREATE TABLE plt2_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_adv_p1 PARTITION OF plt2_adv FOR VALUES IN ('0002'); +CREATE TABLE plt2_adv_p2 PARTITION OF plt2_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt2_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4); +ANALYZE plt2_adv; + +CREATE TABLE plt3_adv (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_adv_p1 PARTITION OF plt3_adv FOR VALUES IN ('0001'); +CREATE TABLE plt3_adv_p2 PARTITION OF plt3_adv FOR VALUES IN ('0003', '0004'); +INSERT INTO plt3_adv SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4); +ANALYZE plt3_adv; + +-- This tests that when merging partitions from plt1_adv and plt2_adv in +-- merge_list_bounds(), process_outer_partition() returns an already-assigned +-- merged partition when re-called with plt1_adv_p1 for the second list value +-- '0001' of that partitin +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a; +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_adv t1 LEFT JOIN plt2_adv t2 ON (t1.c = t2.c)) FULL JOIN plt3_adv t3 ON (t1.c = t3.c) WHERE coalesce(t1.a, 0) % 5 != 3 AND coalesce(t1.a, 0) % 5 != 4 ORDER BY t1.c, t1.a, t2.a, t3.a; + +DROP TABLE plt1_adv; +DROP TABLE plt2_adv; +DROP TABLE plt3_adv; + + +-- Tests for multi-level partitioned tables +CREATE TABLE alpha (a double precision, b int, c text) PARTITION BY RANGE (a); +CREATE TABLE alpha_neg PARTITION OF alpha FOR VALUES FROM ('-Infinity') TO (0) PARTITION BY RANGE (b); +CREATE TABLE alpha_pos PARTITION OF alpha FOR VALUES FROM (0) TO (10.0) PARTITION BY LIST (c); +CREATE TABLE alpha_neg_p1 PARTITION OF alpha_neg FOR VALUES FROM (100) TO (200); +CREATE TABLE alpha_neg_p2 PARTITION OF alpha_neg FOR VALUES FROM (200) TO (300); +CREATE TABLE alpha_neg_p3 PARTITION OF alpha_neg FOR VALUES FROM (300) TO (400); +CREATE TABLE alpha_pos_p1 PARTITION OF alpha_pos FOR VALUES IN ('0001', '0003'); +CREATE TABLE alpha_pos_p2 PARTITION OF alpha_pos FOR VALUES IN ('0004', '0006'); +CREATE TABLE alpha_pos_p3 PARTITION OF alpha_pos FOR VALUES IN ('0008', '0009'); +INSERT INTO alpha_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +INSERT INTO alpha_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 399) i WHERE i % 10 IN (1, 3, 4, 6, 8, 9); +ANALYZE alpha; + +CREATE TABLE beta (a double precision, b int, c text) PARTITION BY RANGE (a); +CREATE TABLE beta_neg PARTITION OF beta FOR VALUES FROM (-10.0) TO (0) PARTITION BY RANGE (b); +CREATE TABLE beta_pos PARTITION OF beta FOR VALUES FROM (0) TO ('Infinity') PARTITION BY LIST (c); +CREATE TABLE beta_neg_p1 PARTITION OF beta_neg FOR VALUES FROM (100) TO (150); +CREATE TABLE beta_neg_p2 PARTITION OF beta_neg FOR VALUES FROM (200) TO (300); +CREATE TABLE beta_neg_p3 PARTITION OF beta_neg FOR VALUES FROM (350) TO (500); +CREATE TABLE beta_pos_p1 PARTITION OF beta_pos FOR VALUES IN ('0002', '0003'); +CREATE TABLE beta_pos_p2 PARTITION OF beta_pos FOR VALUES IN ('0004', '0006'); +CREATE TABLE beta_pos_p3 PARTITION OF beta_pos FOR VALUES IN ('0007', '0009'); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_neg SELECT -1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(100, 149) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(200, 299) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +INSERT INTO beta_pos SELECT 1.0, i, to_char(i % 10, 'FM0000') FROM generate_series(350, 499) i WHERE i % 10 IN (2, 3, 4, 6, 7, 9); +ANALYZE beta; + +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b; +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b) WHERE t1.b >= 125 AND t1.b < 225 ORDER BY t1.a, t1.b; + +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b; +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b, t2.b; + +EXPLAIN (COSTS OFF) +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b; +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.b = t2.b AND t1.c = t2.c) WHERE ((t1.b >= 100 AND t1.b < 110) OR (t1.b >= 200 AND t1.b < 210)) AND ((t2.b >= 100 AND t2.b < 110) OR (t2.b >= 200 AND t2.b < 210)) AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b; -- 2.21.1