diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c index d76fae44b8..e42a52fb9b 100644 --- a/src/backend/nodes/outfuncs.c +++ b/src/backend/nodes/outfuncs.c @@ -2285,6 +2285,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..77e6ff5376 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 partition-wise 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,107 @@ 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 + * create it along with pairs of partitions to be joined; else generate + * those using the partitioning info for the join rel we already have. */ - 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: partitions with the same + * partition indexes will form join pairs, and the join rel will have + * the same partition bounds as inputs; otherwise try to merge the + * partition bounds along with generating join pairs. + * + * Even if one or both inputs have merged partition bounds, it'd be + * possible for the partition bounds to be exactly the same, but it + * seems unlikely to be worth the cycles to check; do this check only + * if both inputs have non-merged partition bounds. + */ + 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 + { + boundinfo = partition_bounds_merge(part_scheme->partnatts, + part_scheme->parttyplen, + part_scheme->parttypbyval, + 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 partition bounds for the join rel are not merged ones, + * inputs are guaranteed to have the same partition bounds, so + * partitions with the same partition indexes will form join pairs; + * else let get_matching_part_pairs() do the work. + */ + 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 +1506,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 +1619,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 +1837,99 @@ match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op) return -1; } + +/* + * get_matching_part_pairs + * Generate join pairs of partitions for 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 those partitions should be considered as ones to be + * joined even 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..1df6da64ed 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 @@ -1681,17 +1669,20 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, /* * Join relation is partitioned using the same partitioning scheme as the - * joining relations and has same bounds. + * joining relations. + * + * Because of restrictions in partition_bounds_merge(), not every pair of + * joining relations (including the one presented to this function) for the + * same joinrel can use partition-wise join or has both the relations + * partitioned. Hence we calculate the partition bounds, number of + * partitions and child-join relations of the join relation when and if we + * find a suitable pair in try_partition_wise_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 95af37b9c7..3c7b6030c0 100644 --- a/src/backend/partitioning/partbounds.c +++ b/src/backend/partitioning/partbounds.c @@ -25,6 +25,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,23 @@ typedef struct PartitionRangeBound bool lower; /* this is the lower (vs upper) bound */ } PartitionRangeBound; +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 remap partitions? */ + int *old_indexes; /* old indexes of merged partitions if + * did_remapping */ +} PartitionMap; + +#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); @@ -107,6 +125,104 @@ static void get_range_key_properties(PartitionKey key, int keynum, Expr **keyCol, Const **lower_val, Const **upper_val); static List *get_range_nulltest(PartitionKey key); +static PartitionBoundInfo partition_range_bounds_merge(int partnatts, FmgrInfo *partsupfuncs, + Oid *partcollations, + RelOptInfo *outer_rel, RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, List **inner_parts); +static PartitionBoundInfo partition_list_bounds_merge(FmgrInfo *partsupfunc, Oid *collations, + 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 map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map, + int outer_part, int inner_part, int *next_index); +static int merge_partition_with_dummy(PartitionMap *map, int index, + 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 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_part_list, + List **inner_part_list); +static PartitionBoundInfo build_merged_partition_bounds(char strategy, + List *merged_datums, List *merged_indexes, + List *merged_contents, int null_index, + int default_index); +static int get_range_partition(RelOptInfo *rel, + PartitionBoundInfo bi, + int *lb_index, + 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 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 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); /* * get_qual_from_partbound @@ -2999,3 +3115,1599 @@ satisfies_hash_partition(PG_FUNCTION_ARGS) PG_RETURN_BOOL(rowHash % modulus == remainder); } + +/* + * 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, + int16 *parttyplen, bool *parttypbyval, + FmgrInfo *partsupfunc, Oid *partcollation, + RelOptInfo *outer_rel, RelOptInfo *inner_rel, + JoinType jointype, List **outer_parts, + List **inner_parts) +{ + PartitionBoundInfo merged_bounds; + PartitionBoundInfo outer_binfo = outer_rel->boundinfo, + inner_binfo = inner_rel->boundinfo; + char strategy = outer_binfo->strategy; + + /* Bail out if partitioning strategies are different. */ + if (outer_binfo->strategy != inner_binfo->strategy) + return NULL; + + if (jointype != JOIN_LEFT && jointype != JOIN_INNER && + jointype != JOIN_SEMI && jointype != JOIN_ANTI && + jointype != JOIN_FULL) + elog(ERROR, "unexpected join type %d", jointype); + + *outer_parts = NIL; + *inner_parts = NIL; + switch (strategy) + { + case PARTITION_STRATEGY_HASH: + merged_bounds = NULL; + + break; + + case PARTITION_STRATEGY_LIST: + merged_bounds = partition_list_bounds_merge(partsupfunc, + partcollation, + outer_rel, + inner_rel, + jointype, + outer_parts, + inner_parts); + break; + + case PARTITION_STRATEGY_RANGE: + merged_bounds = partition_range_bounds_merge(partnatts, + partsupfunc, + partcollation, + outer_rel, + inner_rel, + jointype, + outer_parts, + inner_parts); + break; + + default: + elog(ERROR, "unexpected partition strategy: %d", strategy); + } + + Assert(merged_bounds || (*outer_parts == NIL && *inner_parts == NIL)); + + Assert(list_length(*outer_parts) == list_length(*inner_parts)); + + return merged_bounds; +} + +/* + * get_range_partition + * Returns the next non-dummy partition of a given range-partitioned + * relation + * + * *lb and *ub are set to the lower and upper bounds of that partition + * respectively, and *lb_index is advanced to the next lower bound, if any. + */ +static int +get_range_partition(RelOptInfo *rel, + PartitionBoundInfo bi, + int *lb_index, + PartitionRangeBound *lb, + PartitionRangeBound *ub) +{ + int part_index; + + Assert(bi->strategy == PARTITION_STRATEGY_RANGE); + + do { + part_index = get_range_partition_internal(bi, lb_index, 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_index, + PartitionRangeBound *lb, + PartitionRangeBound *ub) +{ + /* Return the index as -1 if we've exhausted all the lower bounds. */ + if (*lb_index >= bi->ndatums) + return -1; + + /* A lower bound should have at least one more bound after it. */ + Assert(*lb_index + 1 < bi->ndatums); + + lb->index = bi->indexes[*lb_index]; + lb->kind = bi->kind[*lb_index]; + lb->datums = bi->datums[*lb_index]; + lb->lower = true; + ub->index = bi->indexes[*lb_index + 1]; + ub->kind = bi->kind[*lb_index + 1]; + ub->datums = bi->datums[*lb_index + 1]; + ub->lower = false; + + /* The partition index of an upper bound should be valid. */ + Assert(ub->index >= 0); + + /* + * Advance the lower bound; if there are no bounds left beyond the upper + * bound, we have reached the last lower bound. + */ + if (*lb_index + 2 >= bi->ndatums) + *lb_index = bi->ndatums; + else + { + /* + * If the index assigned to the bound next to the upper bound isn't + * valid, that is the lower bound of the next range partition; else, + * the upper bound of the current range partition is also the lower + * bound of the next range partition. + */ + if (bi->indexes[*lb_index + 2] < 0) + *lb_index = *lb_index + 2; + else + *lb_index = *lb_index + 1; + } + + return ub->index; +} + +/* + * compare_range_partitions + * Compares the bounds of two range partitions, and returns true if the + * ranges of the 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 to see if the upper bound of the outer partition is lower than + * the lower bound of the inner partition; in which case 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 to see if the lower bound of the outer partition is higher than + * the upper bound of the inner partition; in which case 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 join, determine the range + * bounds of the 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); + + /* + * An outer join will have all the rows from the outer side, so merged + * bounds will be same as the outer bounds. An inner join will have rows + * that fit both the bounds, thus lower merged bound will be higher of two + * lower bounds and upper merged bound will be lower of the two upper + * bounds. + */ + switch (jointype) + { + case JOIN_INNER: + case JOIN_SEMI: + *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: + *merged_ub = *outer_ub; + *merged_lb = *outer_lb; + break; + + case JOIN_FULL: + *merged_lb = (lb_cmpval < 0) ? *outer_lb : *inner_lb; + *merged_ub = (ub_cmpval > 0) ? *outer_ub : *inner_ub; + break; + + default: + elog(ERROR, "unexpected join type %d", jointype); + } +} + +/* + * add_merged_range_bounds + * Add the range 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 && !*merged_indexes); + cmpval = 1; + } + else + { + PartitionRangeBound prev_ub; + + Assert(*merged_kinds && *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); +} + +/* + * partition_range_bounds_merge + * + * partition_bounds_merge()'s arm for range partitioned tables. + */ +static PartitionBoundInfo +partition_range_bounds_merge(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); + int outer_default = outer_bi->default_index; + bool inner_has_default = partition_bound_has_default(inner_bi); + int inner_default = inner_bi->default_index; + PartitionMap outer_map; + PartitionMap inner_map; + int outer_part; + int inner_part; + int outer_lb_index; + int inner_lb_index; + 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_bi->strategy == inner_bi->strategy && + outer_bi->strategy == PARTITION_STRATEGY_RANGE); + + Assert(*outer_parts == NIL); + Assert(*inner_parts == NIL); + + 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 the ranges (partitions) from both sides. Every iteration compares + * a pair of ranges, one from each side, advancing to the next range from + * the side with smaller upper range bound. If upper bounds of ranges from + * both sides match exactly, both the sides are advanced. For a given pair + * of ranges, we decide whether the corresponding partition match or not. + * lb_index, for inner or outer side, keeps track of the index of lower bound + * datum in PartitionBoundInfo::datums of that side. + */ + outer_lb_index = inner_lb_index = 0; + outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index, + &outer_lb, &outer_ub); + inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index, + &inner_lb, &inner_ub); + while (outer_part >= 0 || inner_part >= 0) + { + PartitionRangeBound merged_lb; + PartitionRangeBound merged_ub; + int merged_index = -1; + bool overlap; + int ub_cmpval; + int lb_cmpval; + + if (outer_part >= 0) + Assert(outer_map.merged_indexes[outer_part] == -1 && + outer_map.merged[outer_part] == false); + if (inner_part >= 0) + Assert(inner_map.merged_indexes[inner_part] == -1 && + inner_map.merged[inner_part] == false); + + /* + * We run this loop till both the sides finish. This allows to avoid + * duplicating code to handle the remaining partitions on the side + * which finishes later. For that we set the comparison parameters + * overlap, ub_cmpval and lb_cmpval in such a way that it appears as if + * the side which finishes earlier has an extra partition with lower + * and upper bounds higher than any other partition of the unfinished + * side. That way we advance the partitions on that side till all of + * them are exhausted. + */ + if (outer_part == -1) + { + overlap = false; + lb_cmpval = 1; + ub_cmpval = 1; + } + else if (inner_part == -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) + { + /* The ranges of partitions overlap; form a join pair of them. */ + + PartitionRangeBound save_outer_ub; + PartitionRangeBound save_inner_ub; + + /* Get the bounds 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); + + /* + * Get the index of the merged partition. Both partitions aren't + * merged yet, so the partitions should be merged successfully. + */ + merged_index = map_and_merge_partitions(&outer_map, &inner_map, + outer_part, inner_part, + &next_index); + Assert(merged_index >= 0); + + /* 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 partitions. */ + outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index, + &outer_lb, &outer_ub); + inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index, + &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_part >= 0 && + compare_range_bounds(partnatts, partsupfuncs, partcollations, + &save_outer_ub, &inner_lb) > 0) + goto cleanup; + if (ub_cmpval < 0 && outer_part >= 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; if so, give up. + */ + 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) + { + /* Upper bound of inner range higher than that of the outer. */ + + merged_lb = outer_lb; + merged_ub = outer_ub; + + if (inner_has_default || IS_OUTER_JOIN(jointype)) + { + merged_index = process_outer_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + outer_part, + inner_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + } + + /* Move to the next partition on the outer side. */ + outer_part = get_range_partition(outer_rel, outer_bi, &outer_lb_index, + &outer_lb, &outer_ub); + } + else + { + /* Upper bound of outer range higher than that of the inner. */ + Assert(ub_cmpval > 0); + + merged_lb = inner_lb; + merged_ub = inner_ub; + + if (outer_has_default || jointype == JOIN_FULL) + { + merged_index = process_inner_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + inner_part, + outer_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + } + + /* Move to the next partition on the inner side. */ + inner_part = get_range_partition(inner_rel, inner_bi, &inner_lb_index, + &inner_lb, &inner_ub); + } + + if (merged_index >= 0 && merged_index != default_index) + { + /* Add the range bounds of the merged partition. */ + add_merged_range_bounds(partnatts, partsupfuncs, partcollations, + &merged_lb, &merged_ub, merged_index, + &merged_datums, &merged_kinds, + &merged_indexes); + } + } + + /* Merge 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); + + /* + * Unlike the case for list partitioning, we wouldn't have re-merged + * partitions, so did_remapping should be left alone. + */ + Assert(!outer_map.did_remapping && !inner_map.did_remapping); + + /* Use maps to match partition from the joining relations. */ + generate_matching_part_pairs(outer_rel, inner_rel, + &outer_map, &inner_map, + next_index, + outer_parts, inner_parts); + + /* Craft a PartitionBoundInfo to return. */ + if (*outer_parts && *inner_parts) + { + Assert(list_length(*outer_parts) == list_length(*inner_parts)); + Assert(list_length(*outer_parts) == next_index); + merged_bounds = build_merged_partition_bounds(outer_bi->strategy, + merged_datums, + merged_indexes, + merged_kinds, + -1, default_index); + } + +cleanup: + /* Free local memory before returning. */ + free_partition_map(&outer_map); + free_partition_map(&inner_map); + list_free(merged_datums); + list_free(merged_indexes); + list_free(merged_kinds); + + return merged_bounds; +} + +/* + * partition_list_bounds_merge + * + * partition_bounds_merge()'s arm for list partitioned tables. + * + */ +static PartitionBoundInfo +partition_list_bounds_merge(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); + int outer_default = outer_bi->default_index; + bool inner_has_default = partition_bound_has_default(inner_bi); + 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 next_index = 0; + int null_index = -1; + int default_index = -1; + List *merged_datums = NIL; + List *merged_indexes = NIL; + int cnto; + int cnti; + + Assert(*outer_parts == NIL); + Assert(*inner_parts == NIL); + + Assert(outer_bi->strategy == inner_bi->strategy && + outer_bi->strategy == PARTITION_STRATEGY_LIST); + + /* List partitions do not require unbounded ranges. */ + 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 the list value datums from both sides. Every iteration compares a + * pair of datums, one from each side, advancing to the next datum from the + * side with smaller datum. If datums from both sides match exactly, both + * the sides are advanced. For a given pair of datums, we decide whether + * the corresponding partition match or not. + */ + cnto = cnti = 0; + while (cnto < outer_bi->ndatums || cnti < inner_bi->ndatums) + { + Datum *merged_datum = NULL; + int merged_index = -1; + int o_index = -1; + int i_index = -1; + Datum *odatums; + Datum *idatums; + int cmpval; + + if (cnto < 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. + */ + o_index = outer_bi->indexes[cnto]; + if (is_dummy_partition(outer_rel, o_index)) + { + cnto++; + continue; + } + } + if (cnti < 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. + */ + i_index = inner_bi->indexes[cnti]; + if (is_dummy_partition(inner_rel, i_index)) + { + cnti++; + continue; + } + } + + /* Get the list datums of the next pair of partitions. */ + odatums = cnto < outer_bi->ndatums ? outer_bi->datums[cnto] : NULL; + idatums = cnti < inner_bi->ndatums ? inner_bi->datums[cnti] : NULL; + + /* + * We run this loop till both the sides finish. This allows to avoid + * duplicating code to handle the remaining datums 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 datum higher than any other datum on the unfinished side. + * That way we advance the datums on the unfinished side till all of + * its datums are exhausted. + */ + if (cnto >= outer_bi->ndatums) + cmpval = 1; + else if (cnti >= inner_bi->ndatums) + cmpval = -1; + else + { + Assert(odatums != NULL && idatums != NULL); + cmpval = DatumGetInt32(FunctionCall2Coll(&partsupfunc[0], + partcollation[0], + odatums[0], + idatums[0])); + } + + if (cmpval == 0) + { + /* + * Datums match. Rows on either side with these datums as partition + * key value will join and will be part of the partition of the + * join result produced by joining the corresponding partitions. + * Match the corresponding partitions and if successful, add the + * datum to the list of merged datums with index of merged + * partition containing it. + */ + merged_datum = odatums; + Assert(o_index >= 0 && i_index >= 0); + merged_index = map_and_merge_partitions(&outer_map, &inner_map, + o_index, i_index, + &next_index); + + if (merged_index == -1) + goto cleanup; + + /* Move to the next pair of bounds. */ + cnto++; + cnti++; + } + else if (cmpval < 0) + { + Assert(cnto < outer_bi->ndatums); + Assert(o_index >= 0); + + /* A datum missing from the inner side. */ + merged_datum = odatums; + + if (inner_has_default || IS_OUTER_JOIN(jointype)) + { + Assert(o_index >= 0); + merged_index = process_outer_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + o_index, + inner_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + } + + /* Move to the next datum on the outer side. */ + cnto++; + } + else + { + Assert(cmpval > 0); + Assert(cnti < inner_bi->ndatums); + Assert(i_index >= 0); + + /* A datum missing from the outer side. */ + merged_datum = idatums; + + if (outer_has_default || jointype == JOIN_FULL) + { + Assert(i_index >= 0); + merged_index = process_inner_partition(&outer_map, + &inner_map, + outer_has_default, + inner_has_default, + i_index, + outer_default, + jointype, + &next_index, + &default_index); + if (merged_index == -1) + goto cleanup; + } + + /* Move to the next datum on the inner side. */ + cnti++; + } + + /* + * Add the list value with appropriate index in the list of datums, if + * we have associated a partition with this list value. + */ + if (merged_index >= 0 && merged_index != default_index) + { + merged_indexes = lappend_int(merged_indexes, merged_index); + merged_datums = lappend(merged_datums, merged_datum); + } + } + + /* + * 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 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 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); + + /* 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 partition from the joining relations. */ + generate_matching_part_pairs(outer_rel, inner_rel, + &outer_map, &inner_map, + next_index, + outer_parts, inner_parts); + + /* Craft a PartitionBoundInfo to return. */ + if (*outer_parts && *inner_parts) + { + Assert(list_length(*outer_parts) == list_length(*inner_parts)); + Assert(list_length(*outer_parts) <= next_index); + merged_bounds = build_merged_partition_bounds(outer_bi->strategy, + merged_datums, + merged_indexes, NIL, + null_index, default_index); + } + +cleanup: + /* Free local memory before returning. */ + free_partition_map(&outer_map); + free_partition_map(&inner_map); + list_free(merged_datums); + list_free(merged_indexes); + + 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; +} + +/* + * map_and_merge_partitions + * + * *next_index is incremented when creating a new merged partition associated + * with the given partitions. + */ +static int +map_and_merge_partitions(PartitionMap *outer_map, PartitionMap *inner_map, + int outer_index, int inner_index, int *next_index) +{ + int outer_merged_index; + bool outer_merged; + int inner_merged_index; + 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 both partitions are mapped to merged 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 partition. Otherwise, if both + * partitions are merged with dummy partitions, re-merge them; map + * them to the merged partition with the smaller of the two merged + * indexes and return the smaller index. 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) + { + /* + * Note that we will fix the larger index that have been added to + * the merged_indexes list so far in fix_merged_indexes(). + */ + 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 partition isn't mapped to a merged partition. */ + Assert(outer_merged_index == -1 || inner_merged_index == -1); + + /* + * If neither of partitions isn't mapped, assign them a new merged + * partition and return the index of the merged partition. Otherwise, if + * one of partitions is merged with a dummy relation (and the other isn't + * merged), re-merge it with the other, with the same index, and return + * the index. 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; +} + +/* + * merge_partition_with_dummy + * + * *next_index is incremented. + */ +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; +} + +/* + * process_outer_partition + * + * Determine the merged partition associated with the given outer partition. + * + * *next_index is incremented when creating a new merged partition associated + * with the given outer partition. + */ +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, the outer partition has to + * be joined with the default partition; try merging them. Otherwise, we + * should in an outer join, in which case the outer partition has to be + * scanned all the way anyway; if the outer partition is already mapped to + * a merged partition, get it, otherwise create a new merged partition by + * merging the outer partition with a dummy partition. + */ + if (inner_has_default) + { + Assert(inner_default >= 0); + + /* + * If the outer side has the default partition as well, we need to + * merge the default partitions (see merge_default_partitions()); give + * up on it. + */ + if (outer_has_default) + return -1; + + merged_index = map_and_merge_partitions(outer_map, inner_map, + outer_index, inner_default, + next_index); + if (merged_index == -1) + return -1; + + /* + * If this is a FULL join, the merged partition would act as the + * default partition of the join; record the index in *default_index + * if not done yet. + */ + 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); + + 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 + * + * Determine the merged partition associated with the given inner partition. + * + * *next_index is incremented when creating a new merged partition associated + * with the given inner partition. + */ +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, the inner partition has to + * be joined with the default partition; try merging them. Otherwise, we + * should in an FULL join, in which case the inner partition has to be + * scanned all the way anyway; if the inner partition is already mapped to + * a merged partition, get it, otherwise create a new merged partition by + * merging the inner partition with a dummy partition. + */ + if (outer_has_default) + { + Assert(outer_default >= 0); + + /* + * If the inner side has the default partition as well, we need to + * merge the default partitions (see merge_default_partitions()); give + * up on it. + */ + if (inner_has_default) + return -1; + + merged_index = map_and_merge_partitions(outer_map, inner_map, + outer_default, inner_index, + next_index); + if (merged_index == -1) + return -1; + + /* + * If this is an outer join, the merged partition would act as the + * default partition of the join; record the index in *default_index + * if not done yet. + */ + 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); + + 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; +} + +/* + * fix_merged_indexes + */ +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; + + 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 + * + * This function produces the list pairs of partitions which produce merged + * partitions in the order of merged partition indexes. + * + * nmerged is the number of merged partitions. + * + * If successful, the list pairs are returned as two separate lists, + * outer_part_list and inner_part_list, one for each side. Otherwise, those + * lists will be set to NIL. + */ +static void +generate_matching_part_pairs(RelOptInfo *outer_rel, RelOptInfo *inner_rel, + PartitionMap *outer_map, PartitionMap *inner_map, + int nmerged, + List **outer_part_list, List **inner_part_list) +{ + int outer_nparts = outer_map->nparts; + int inner_nparts = inner_map->nparts; + int *outer_part_array; + int *inner_part_array; + int max_nparts; + int i; + + Assert(outer_nparts == outer_rel->nparts); + Assert(inner_nparts == inner_rel->nparts); + + outer_part_array = (int *) palloc(sizeof(int) * nmerged); + inner_part_array = (int *) palloc(sizeof(int) * nmerged); + for (i = 0; i < nmerged; i++) + outer_part_array[i] = inner_part_array[i] = -1; + + /* Set pairs of matching partitions. */ + 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_part_array[merged_index] = i; + } + } + if (i < inner_nparts) + { + int merged_index = inner_map->merged_indexes[i]; + + if (merged_index >= 0) + { + Assert(merged_index < nmerged); + inner_part_array[merged_index] = i; + } + } + } + + *outer_part_list = NIL; + *inner_part_list = NIL; + + /* Build the list pairs. */ + for (i = 0; i < nmerged; i++) + { + int outer_part = outer_part_array[i]; + int inner_part = inner_part_array[i]; + + /* + * If both partitions are dummy, it means the merged partition was + * removed by partition re-merging in map_and_merge_partitions(); + * ignore the merged partition + */ + if (outer_part == -1 && inner_part == -1) + continue; + + *outer_part_list = lappend(*outer_part_list, outer_part >= 0 ? + outer_rel->part_rels[outer_part] : NULL); + *inner_part_list = lappend(*inner_part_list, inner_part >= 0 ? + inner_rel->part_rels[inner_part] : NULL); + } + + pfree(outer_part_array); + pfree(inner_part_array); +} + +static PartitionBoundInfo +build_merged_partition_bounds(char strategy, List *merged_datums, + List *merged_indexes, List *merged_kinds, + int null_index, int default_index) +{ + int cnt; + PartitionBoundInfo merged_bounds; + ListCell *lc; + + /* We expect the same number of elements in datums and indexes lists. */ + Assert(list_length(merged_datums) == list_length(merged_indexes)); + + merged_bounds = (PartitionBoundInfo) palloc(sizeof(PartitionBoundInfoData)); + merged_bounds->strategy = strategy; + merged_bounds->ndatums = list_length(merged_datums); + + if (strategy == PARTITION_STRATEGY_RANGE) + { + Assert(list_length(merged_datums) == list_length(merged_kinds)); + merged_bounds->kind = + (PartitionRangeDatumKind **) palloc(sizeof(PartitionRangeDatumKind *) * + list_length(merged_kinds)); + cnt = 0; + foreach(lc, merged_kinds) + merged_bounds->kind[cnt++] = lfirst(lc); + + /* There are ndatums+1 indexes in case of range partitions */ + merged_indexes = lappend_int(merged_indexes, -1); + } + else + merged_bounds->kind = NULL; + + cnt = 0; + merged_bounds->datums = (Datum **) palloc(sizeof(Datum *) * + list_length(merged_datums)); + foreach(lc, merged_datums) + merged_bounds->datums[cnt++] = lfirst(lc); + + merged_bounds->indexes = (int *) palloc(sizeof(int) * + list_length(merged_indexes)); + cnt = 0; + foreach(lc, merged_indexes) + merged_bounds->indexes[cnt++] = lfirst_int(lc); + + merged_bounds->null_index = null_index; + merged_bounds->default_index = default_index; + + return merged_bounds; +} + +/* + * merge_default_partitions + * Merge the default partitions from both sides, if any + * + * *default_index is set to the index of the default partition of the joinrel, + * if appropriate. + */ +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); + + 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, assign a new merged partition if not + * already done. The resulting partition should be the default + * partition of the joinrel. + */ + if (IS_OUTER_JOIN(jointype)) + { + Assert(jointype != JOIN_RIGHT); + if (outer_merged_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, assign a new merged partition if not + * already done. The resulting partition should be the default + * partition of the joinrel. + */ + if (jointype == JOIN_FULL) + { + if (inner_merged_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); + Assert(outer_merged_index == -1); + Assert(inner_merged_index == -1); + Assert(*default_index == -1); + + /* + * Merge the default partitions. Both partitions aren't merged yet, + * so the partitions should be merged successfully. The resulting + * partition should be the default partition of the joinrel. + */ + *default_index = map_and_merge_partitions(outer_map, + inner_map, + outer_default, + inner_default, + next_index); + Assert(*default_index >= 0); + } +} + +/* + * merge_null_partitions + * Merge the NULL partitions from both sides, if any + * + * *null_index is set to the index of the NULL partition of the joinrel, if + * appropriate. + */ +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 outer_null_unmerged = false; + bool inner_null_unmerged = false; + + Assert(outer_has_null || inner_has_null); + Assert(*null_index == -1); + + /* + * If this is an INNER or SEMI join, two NULL values won't be considered + * to be equal, in which case we don't need to do anything. + */ + if (jointype == JOIN_INNER || jointype == JOIN_SEMI) + return; + + Assert(IS_OUTER_JOIN(jointype)); + Assert(jointype != JOIN_RIGHT); + + if (outer_has_null) + { + /* + * Since this is an outer join, if the NULL partition of the outer + * side isn't merged yet, it means that the partition contains only + * NULL values as the key values, in which case we need to create a + * merged partition below. + */ + Assert(outer_null >= 0 && outer_null < outer_map->nparts); + if (outer_map->merged_indexes[outer_null] == -1) + outer_null_unmerged = true; + } + if (inner_has_null) + { + /* + * If this is a FULL join, and the NULL partition of the inner side + * isn't merged yet, it means that the partition contains only NULL + * values as the key values, in which case we need to create a merged + * partition below. + */ + Assert(inner_null >= 0 && inner_null < inner_map->nparts); + if (inner_map->merged_indexes[inner_null] == -1) + inner_null_unmerged = true; + } + + if (!outer_null_unmerged && !inner_null_unmerged) + return; + + if (outer_null_unmerged && !inner_null_unmerged) + { + Assert(outer_has_null); + + /* + * Assign a new merged partition. The resulting partition should be + * the NULL partition of the joinrel. + */ + *null_index = merge_partition_with_dummy(outer_map, outer_null, + next_index); + } + else if (!outer_null_unmerged && inner_null_unmerged) + { + Assert(inner_has_null); + + /* + * Assign a new merged partition. The resulting partition should be + * the NULL partition of the joinrel. + */ + if (jointype == JOIN_FULL) + *null_index = merge_partition_with_dummy(inner_map, inner_null, + next_index); + } + else + { + Assert(outer_null_unmerged && inner_null_unmerged); + Assert(outer_has_null); + Assert(inner_has_null); + + /* + * Merge the NULL partitions. Both partitions aren't merged yet, so + * the partitions should be merged successfully. The resulting + * partition should be the NULL partition of the joinrel. + */ + *null_index = map_and_merge_partitions(outer_map, inner_map, + outer_null, inner_null, + next_index); + Assert(*null_index >= 0); + } +} diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h index 3d3be197e0..1545877d8c 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 @@ -720,9 +722,12 @@ typedef struct RelOptInfo PartitionScheme part_scheme; /* Partitioning scheme. */ int nparts; /* number of partitions */ 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..5442e843af 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 */ /* @@ -108,5 +109,11 @@ extern int partition_range_datum_bsearch(FmgrInfo *partsupfunc, int nvalues, Datum *values, bool *is_equal); extern int partition_hash_bsearch(PartitionBoundInfo boundinfo, int modulus, int remainder); +extern PartitionBoundInfo partition_bounds_merge(int partnatts, + int16 *parttyplen, bool *parttypbyval, + FmgrInfo *partsupfunc, Oid *partcollation, + struct RelOptInfo *outer_rel, struct RelOptInfo *inner_rel, + JoinType jointype, + List **outer_parts, List **inner_parts); #endif /* PARTBOUNDS_H */ diff --git a/src/test/regress/expected/partition_join.out b/src/test/regress/expected/partition_join.out index b3fbe47bde..8e1aa49da0 100644 --- a/src/test/regress/expected/partition_join.out +++ b/src/test/regress/expected/partition_join.out @@ -2055,30 +2055,2671 @@ 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_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_ad_a_idx on prt1_ad (a); +INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_ad; +CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150); +CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500); +CREATE INDEX prt2_ad_b_idx on prt2_ad (b); +INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i; +INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i; +INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_ad; +-- 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_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash Semi Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash Semi Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + QUERY PLAN +----------------------------------------------- + Append + -> Hash Anti Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash Anti Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash Anti Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p3 t2_3 +(19 rows) + +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + 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_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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_ad.a, prt2_ad.b + -> Append + -> Hash Full Join + Hash Cond: (prt1_ad_1.a = prt2_ad_1.b) + Filter: (((175) = prt1_ad_1.a) OR ((425) = prt2_ad_1.b)) + -> Seq Scan on prt1_ad_p1 prt1_ad_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p1 prt2_ad_1 + Filter: (a = 0) + -> Hash Full Join + Hash Cond: (prt1_ad_2.a = prt2_ad_2.b) + Filter: (((175) = prt1_ad_2.a) OR ((425) = prt2_ad_2.b)) + -> Seq Scan on prt1_ad_p2 prt1_ad_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p2 prt2_ad_2 + Filter: (a = 0) + -> Hash Full Join + Hash Cond: (prt2_ad_3.b = prt1_ad_3.a) + Filter: (((175) = prt1_ad_3.a) OR ((425) = prt2_ad_3.b)) + -> Seq Scan on prt2_ad_p3 prt2_ad_3 + Filter: (a = 0) + -> Hash + -> Seq Scan on prt1_ad_p3 prt1_ad_3 + Filter: (b = 0) +(27 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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 +-- Add an extra partition to prt2_ad +CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i; +ANALYZE prt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash Semi Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash Semi Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p3 t2_3 +(21 rows) + +SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t2_3.b = t1_3.a) + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad 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_ad t1 LEFT JOIN prt1_ad 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_ad_p1 t2_1 + -> Seq Scan on prt1_ad_p2 t2_2 + -> Seq Scan on prt1_ad_p3 t2_3 -> Hash -> Append - -> Seq Scan on prt1_p1 t1_1 + -> Seq Scan on prt2_ad_p1 t1_1 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p2 t1_2 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p3 t1_3 + Filter: (a = 0) + -> Seq Scan on prt2_ad_extra t1_4 + Filter: (a = 0) +(18 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + QUERY PLAN +----------------------------------------------- + Append + -> Hash Anti Join + Hash Cond: (t1_1.a = t2_1.b) + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash Anti Join + Hash Cond: (t1_2.a = t2_2.b) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash Anti Join + Hash Cond: (t1_3.a = t2_3.b) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Seq Scan on prt2_ad_p3 t2_3 +(19 rows) + +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0; + QUERY PLAN +----------------------------------------------- + Hash Anti Join + Hash Cond: (t1.b = t2.a) + -> Append + -> Seq Scan on prt2_ad_p1 t1_1 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p2 t1_2 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p3 t1_3 + Filter: (a = 0) + -> Seq Scan on prt2_ad_extra t1_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_ad_p1 t2_1 + -> Seq Scan on prt1_ad_p2 t2_2 + -> Seq Scan on prt1_ad_p3 t2_3 +(16 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_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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_ad.a, prt2_ad.b + -> Hash Full Join + Hash Cond: (prt2_ad.b = prt1_ad.a) + Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b)) + -> Append + -> Seq Scan on prt2_ad_p1 prt2_ad_1 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p2 prt2_ad_2 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p3 prt2_ad_3 + Filter: (a = 0) + -> Seq Scan on prt2_ad_extra prt2_ad_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_ad_p1 prt1_ad_1 Filter: (b = 0) - -> Seq Scan on prt1_p2 t1_2 + -> Seq Scan on prt1_ad_p2 prt1_ad_2 Filter: (b = 0) - -> Seq Scan on prt1_p3 t1_3 + -> Seq Scan on prt1_ad_p3 prt1_ad_3 Filter: (b = 0) -(16 rows) +(22 rows) + +-- 3-way join where not every pair of relations can do partitioned join +EXPLAIN (COSTS OFF) +SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c; + QUERY PLAN +------------------------------------------------------------------------ + Sort + Sort Key: t1.a, t2.a, t3.c + -> Append + -> Nested Loop + Join Filter: (t1_1.a = t3_1.a) + -> Hash Join + Hash Cond: (t2_1.b = t1_1.a) + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Index Scan using prt1_ad_p1_a_idx on prt1_ad_p1 t3_1 + Index Cond: (a = t2_1.b) + -> Nested Loop + Join Filter: (t1_2.a = t3_2.a) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Index Scan using prt1_ad_p2_a_idx on prt1_ad_p2 t3_2 + Index Cond: (a = t2_2.b) + -> Nested Loop + Join Filter: (t1_3.a = t2_3.b) + -> Hash Join + Hash Cond: (t3_3.a = t1_3.a) + -> Seq Scan on prt1_ad_p3 t3_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Index Scan using prt2_ad_p3_b_idx on prt2_ad_p3 t2_3 + Index Cond: (b = t3_3.a) +(33 rows) + +SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c; + a | a | 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) + +DROP TABLE prt2_ad_extra; +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in those cases +-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both +-- partitions +ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3; +CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375); +CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3_350_375 t2_3 + -> Seq Scan on prt2_ad_p3_375_500 t2_4 + -> Hash + -> Append + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(17 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Append + -> Seq Scan on prt2_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3_350_375 t2_3 + -> Seq Scan on prt2_ad_p3_375_500 t2_4 +(17 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 LEFT JOIN prt2_ad 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_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3_350_375 t2_3 + -> Seq Scan on prt2_ad_p3_375_500 t2_4 + -> Hash + -> Append + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) +(17 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + QUERY PLAN +------------------------------------------------------- + Hash Anti Join + Hash Cond: (t1.a = t2.b) + -> Append + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_3 + Filter: (b = 0) + -> Hash + -> Append + -> Seq Scan on prt2_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3_350_375 t2_3 + -> Seq Scan on prt2_ad_p3_375_500 t2_4 +(15 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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_ad.a, prt2_ad.b + -> Hash Full Join + Hash Cond: (prt2_ad.b = prt1_ad.a) + Filter: (((175) = prt1_ad.a) OR ((425) = prt2_ad.b)) + -> Append + -> Seq Scan on prt2_ad_p1 prt2_ad_1 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p2 prt2_ad_2 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p3_350_375 prt2_ad_3 + Filter: (a = 0) + -> Seq Scan on prt2_ad_p3_375_500 prt2_ad_4 + Filter: (a = 0) + -> Hash + -> Append + -> Seq Scan on prt1_ad_p1 prt1_ad_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p2 prt1_ad_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 prt1_ad_3 + Filter: (b = 0) +(22 rows) + +DROP TABLE prt2_ad_p3_350_375; +DROP TABLE prt2_ad_p3_375_500; +-- Test default partitions +-- Change prt1_ad_p1 to the default partition +ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1; +ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT; +ANALYZE prt1_ad; +CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i; +ANALYZE prt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_3.a) + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_3 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_2.b = t1_1.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_1 + Filter: (b = 0) + -> Hash Join + Hash Cond: (t2_3.b = t1_2.a) + -> Seq Scan on prt2_ad_p3_300_400 t2_3 + -> Hash + -> Seq Scan on prt1_ad_p3 t1_2 + Filter: (b = 0) +(21 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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 + 300 | 0300 | 300 | 0300 + 325 | 0325 | 325 | 0325 + 350 | 0350 | 350 | 0350 + 375 | 0375 | 375 | 0375 +(10 rows) + +DROP TABLE prt2_ad_p3_300_400; +-- Restore prt2_ad_p3 +ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500); +ANALYZE prt2_ad; +-- Partitioned join can't be applied because the default partition of prt1_ad +-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on prt1_ad_p2 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p1 t1_3 + Filter: (b = 0) +(16 rows) + +-- Change prt2_ad_p3 to the default partition +ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3; +ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT; +ANALYZE prt2_ad; +-- Partitioned join can't be applied because the default partition of prt1_ad +-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Seq Scan on prt2_ad_p2 t2_2 + -> Seq Scan on prt2_ad_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on prt1_ad_p2 t1_1 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p3 t1_2 + Filter: (b = 0) + -> Seq Scan on prt1_ad_p1 t1_3 + Filter: (b = 0) +(16 rows) + +DROP TABLE prt1_ad_p3; +ANALYZE prt1_ad; +DROP TABLE prt2_ad_p3; +ANALYZE prt2_ad; +CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt3_ad_a_idx on prt3_ad (a); +INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i; +ANALYZE prt3_ad; +-- 3-way join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad 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_ad_p1 t3_1 + -> Hash + -> Hash Right Join + Hash Cond: (t2_2.b = t1_1.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_1 + Filter: (b = 0) + -> Hash Right Join + Hash Cond: (t3_2.a = t1_2.a) + -> Seq Scan on prt3_ad_p2 t3_2 + -> Hash + -> Hash Right Join + Hash Cond: (t2_1.b = t1_2.a) + -> Seq Scan on prt2_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_2 + Filter: (b = 0) +(23 rows) + +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad 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_ad; +DROP TABLE prt2_ad; +DROP TABLE prt3_ad; +-- Test interaction of partitioned join with partition pruning +CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_ad_a_idx on prt1_ad (a); +INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_ad; +CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400); +CREATE INDEX prt2_ad_b_idx on prt2_ad (b); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_p1 t1_1 + Filter: ((a < 300) AND (b = 0)) + -> Hash Join + Hash Cond: (t2_2.b = t1_2.a) + -> Seq Scan on prt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_p2 t1_2 + Filter: ((a < 300) AND (b = 0)) +(15 rows) + +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p3; +CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT; +ANALYZE prt1_ad; +CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT; +ANALYZE prt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on prt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on prt1_ad_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_ad t1 INNER JOIN prt2_ad 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_ad; +DROP TABLE prt2_ad; +-- Tests for list-partitioned tables +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7); +ANALYZE plt1_ad; +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8); +ANALYZE plt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_p1 t1_1 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + -> Hash + -> Seq Scan on plt2_ad_p3 t2_3 +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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 +-- Add an extra partition to plt2_ad +CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000'); +INSERT INTO plt2_ad_extra VALUES (0, 0, '0000'); +ANALYZE plt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt1_ad 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_ad_p1 t2_1 + -> Seq Scan on plt1_ad_p2 t2_2 + -> Seq Scan on plt1_ad_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on plt2_ad_extra t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p1 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p2 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t1_4 + Filter: (b < 10) +(18 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t2_3 +(18 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad 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_ad_extra t1_1 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p1 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p2 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3 t1_4 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t2_1 + -> Seq Scan on plt1_ad_p2 t2_2 + -> Seq Scan on plt1_ad_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_ad t1 FULL JOIN plt2_ad 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_ad_extra t2_1 + -> Seq Scan on plt2_ad_p1 t2_2 + -> Seq Scan on plt2_ad_p2 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t1_1 + -> Seq Scan on plt1_ad_p2 t1_2 + -> Seq Scan on plt1_ad_p3 t1_3 +(15 rows) + +DROP TABLE plt2_ad_extra; +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in those cases +-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both +-- partitions +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2; +CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004'); +CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6); +ANALYZE plt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_0004 t2_2 + -> Seq Scan on plt2_ad_p2_0006 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(17 rows) + +-- semi join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt2_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_0004 t2_2 + -> Seq Scan on plt2_ad_p2_0006 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 +(17 rows) + +-- left join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_0004 t2_2 + -> Seq Scan on plt2_ad_p2_0006 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(17 rows) + +-- anti join +EXPLAIN (COSTS OFF) +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) + -> Hash + -> Append + -> Seq Scan on plt2_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_0004 t2_2 + -> Seq Scan on plt2_ad_p2_0006 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 +(17 rows) + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_0004 t2_2 + -> Seq Scan on plt2_ad_p2_0006 t2_3 + -> Seq Scan on plt2_ad_p3 t2_4 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t1_1 + -> Seq Scan on plt1_ad_p2 t1_2 + -> Seq Scan on plt1_ad_p3 t1_3 +(15 rows) + +DROP TABLE plt2_ad_p2_0004; +DROP TABLE plt2_ad_p2_0006; +-- Restore plt2_ad_p2 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_ad; +-- Test NULL partitions +-- Change plt1_ad_p1 to the NULL partition +ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1; +CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt1_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; +-- Change plt2_ad_p3 to the NULL partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3; +CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1_null t1_1 + Filter: (b < 10) + -> Hash + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3_null t2_3 +(19 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad_p1_null t1_1 + Filter: (b < 10) + -> Hash + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt2_ad_p3_null t2_3 +(19 rows) + +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_p1_null t1_1 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p3_null t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad_p1_null; +DROP TABLE plt2_ad_p3_null; +-- Restore plt1_ad_p1 +ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003'); +-- Add to plt1_ad the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL); +INSERT INTO plt1_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; +-- Restore plt2_ad_p3 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009'); +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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) + +-- Partitioned join can't be applied because there isn't any partition on the +-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2 t2_2 + -> Seq Scan on plt2_ad_p3 t2_3 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p1 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p2 t1_2 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) + -> Seq Scan on plt1_ad_extra t1_4 + Filter: (b < 10) +(18 rows) + +-- Partitioned join can't be applied because there isn't any partition on the +-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad_p1 t1_1 + -> Seq Scan on plt1_ad_p2 t1_2 + -> Seq Scan on plt1_ad_p3 t1_3 + -> Seq Scan on plt1_ad_extra t1_4 + -> Hash + -> Append + -> Seq Scan on plt2_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2 t2_2 + -> Seq Scan on plt2_ad_p3 t2_3 +(15 rows) + +-- Add to plt2_ad the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_p3 t1_3 + Filter: (b < 10) +(21 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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) + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p3 t2_3 + -> Hash + -> Seq Scan on plt1_ad_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_ad_extra t1_4 + Filter: (b < 10) + -> Seq Scan on plt2_ad_extra t2_4 +(26 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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) + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad_p1 t1_1 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p2 t1_2 + -> Hash + -> Seq Scan on plt2_ad_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_ad_p3 t1_3 + -> Hash + -> Seq Scan on plt2_ad_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_ad_extra t1_4 + -> Hash + -> Seq Scan on plt2_ad_extra t2_4 +(27 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad_extra; +DROP TABLE plt2_ad_extra; +-- Test default partitions +-- Change plt1_ad_p1 to the default partition +ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1; +ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT; +DROP TABLE plt1_ad_p3; +ANALYZE plt1_ad; +DROP TABLE plt2_ad_p3; +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_p2 t1_1 + Filter: (b < 10) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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) + +-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as +-- the key values +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2; +CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6); +ANALYZE plt2_ad; +-- Partitioned join can't be applied because the default partition of plt1_ad +-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_ext t2_2 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p2 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p1 t1_2 + Filter: (b < 10) +(13 rows) + +-- Change plt2_ad_p2_ext to the default partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext; +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT; +ANALYZE plt2_ad; +-- Partitioned join can't be applied because the default partition of plt1_ad +-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1 t2_1 + -> Seq Scan on plt2_ad_p2_ext t2_2 + -> Hash + -> Append + -> Seq Scan on plt1_ad_p2 t1_1 + Filter: (b < 10) + -> Seq Scan on plt1_ad_p1 t1_2 + Filter: (b < 10) +(13 rows) + +DROP TABLE plt2_ad_p2_ext; +-- Restore plt2_ad_p2 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_ad; +CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009'); +INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9); +ANALYZE plt3_ad; +-- 3-way join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad 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_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_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_ad_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_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt1_ad_p1 t1_2 + Filter: (b < 10) +(23 rows) + +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad 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 +-- Change plt2_ad_p1 to the NULL partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1; +CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p1_null t2_1 + -> Hash + -> Seq Scan on plt1_ad_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_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt1_ad_p2 t1_1 + Filter: (b < 10) +(15 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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) + +-- Change the NULL partition to contain only NULL values as the key values +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null; +CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad_p2 t2 + -> Hash + -> Seq Scan on plt1_ad_p2 t1 + Filter: (b < 10) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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 plt2_ad_p1_nullonly; +DROP TABLE plt2_ad_p1_null; +DROP TABLE plt2_ad_p1; +DROP TABLE plt1_ad; +DROP TABLE plt2_ad; +DROP TABLE plt3_ad; +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i; +ANALYZE plt1_ad; +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4); +ANALYZE plt2_ad; +CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001'); +CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4); +ANALYZE plt3_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad 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_ad_p1 t1_1 + -> Hash + -> Seq Scan on plt2_ad_p1 t2_1 + -> Hash + -> Seq Scan on plt3_ad_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_ad_p2 t1_2 + -> Hash + -> Seq Scan on plt2_ad_p2 t2_2 + -> Hash + -> Seq Scan on plt3_ad_p2 t3_2 +(23 rows) + +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad 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_ad; +DROP TABLE plt2_ad; +DROP TABLE plt3_ad; +-- Test interaction of partitioned join with partition pruning +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002'); +CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003'); +CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005'); +INSERT INTO plt1_ad 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_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003'); +CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL); +CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005'); +INSERT INTO plt2_ad 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_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL 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_ad_p2 t2_1 + Filter: (c IS NOT NULL) + -> Hash + -> Seq Scan on plt1_ad_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_ad_p4 t2_2 + Filter: (c IS NOT NULL) + -> Hash + -> Seq Scan on plt1_ad_p4 t1_2 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) +(17 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL 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_ad t1 LEFT JOIN plt2_ad 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_ad_p4 t2 + -> Hash + -> Seq Scan on plt1_ad_p4 t1 + Filter: ((c IS NULL) AND (b < 10)) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad_default PARTITION OF plt1_ad DEFAULT; +ANALYZE plt1_ad; +CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT; +ANALYZE plt2_ad; +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL 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_ad_p2 t2_1 + Filter: (c IS NOT NULL) + -> Hash + -> Seq Scan on plt1_ad_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_ad_p4 t2_2 + Filter: (c IS NOT NULL) + -> Hash + -> Seq Scan on plt1_ad_p4 t1_2 + Filter: ((b < 10) AND (c = ANY ('{0003,0004,0005}'::text[]))) +(17 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL 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_ad t1 LEFT JOIN plt2_ad 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_ad_p4 t2 + -> Hash + -> Seq Scan on plt1_ad_p4 t1 + Filter: ((c IS NULL) AND (b < 10)) +(8 rows) + +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad; +DROP TABLE plt2_ad; +-- 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 >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b; + QUERY PLAN +------------------------------------------------------------------------------------------- + Sort + Sort Key: t1.a, t1.b, t2.b + -> Append + -> Hash Join + Hash Cond: (t1_2.a = t2_2.a) + -> Append + -> Seq Scan on alpha_neg_p1 t1_2 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) + -> Seq Scan on alpha_neg_p2 t1_3 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) + -> Hash + -> Append + -> Seq Scan on beta_neg_p1 t2_2 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) + -> Seq Scan on beta_neg_p2 t2_3 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) + -> Hash Join + Hash Cond: (t1_4.a = t2_4.a) + -> Seq Scan on alpha_pos_p1 t1_4 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) + -> Hash + -> Seq Scan on beta_pos_p1 t2_4 + Filter: ((b >= 125) AND (b < 225) AND (c = '0003'::text)) +(23 rows) + +SELECT t1.*, t2.* FROM alpha t1 INNER JOIN beta t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, t2.b; + a | b | c | a | b | c +----+-----+------+----+-----+------ + -1 | 133 | 0003 | -1 | 133 | 0003 + -1 | 133 | 0003 | -1 | 143 | 0003 + -1 | 133 | 0003 | -1 | 203 | 0003 + -1 | 133 | 0003 | -1 | 213 | 0003 + -1 | 133 | 0003 | -1 | 223 | 0003 + -1 | 143 | 0003 | -1 | 133 | 0003 + -1 | 143 | 0003 | -1 | 143 | 0003 + -1 | 143 | 0003 | -1 | 203 | 0003 + -1 | 143 | 0003 | -1 | 213 | 0003 + -1 | 143 | 0003 | -1 | 223 | 0003 + -1 | 153 | 0003 | -1 | 133 | 0003 + -1 | 153 | 0003 | -1 | 143 | 0003 + -1 | 153 | 0003 | -1 | 203 | 0003 + -1 | 153 | 0003 | -1 | 213 | 0003 + -1 | 153 | 0003 | -1 | 223 | 0003 + -1 | 163 | 0003 | -1 | 133 | 0003 + -1 | 163 | 0003 | -1 | 143 | 0003 + -1 | 163 | 0003 | -1 | 203 | 0003 + -1 | 163 | 0003 | -1 | 213 | 0003 + -1 | 163 | 0003 | -1 | 223 | 0003 + -1 | 173 | 0003 | -1 | 133 | 0003 + -1 | 173 | 0003 | -1 | 143 | 0003 + -1 | 173 | 0003 | -1 | 203 | 0003 + -1 | 173 | 0003 | -1 | 213 | 0003 + -1 | 173 | 0003 | -1 | 223 | 0003 + -1 | 183 | 0003 | -1 | 133 | 0003 + -1 | 183 | 0003 | -1 | 143 | 0003 + -1 | 183 | 0003 | -1 | 203 | 0003 + -1 | 183 | 0003 | -1 | 213 | 0003 + -1 | 183 | 0003 | -1 | 223 | 0003 + -1 | 193 | 0003 | -1 | 133 | 0003 + -1 | 193 | 0003 | -1 | 143 | 0003 + -1 | 193 | 0003 | -1 | 203 | 0003 + -1 | 193 | 0003 | -1 | 213 | 0003 + -1 | 193 | 0003 | -1 | 223 | 0003 + -1 | 203 | 0003 | -1 | 133 | 0003 + -1 | 203 | 0003 | -1 | 143 | 0003 + -1 | 203 | 0003 | -1 | 203 | 0003 + -1 | 203 | 0003 | -1 | 213 | 0003 + -1 | 203 | 0003 | -1 | 223 | 0003 + -1 | 213 | 0003 | -1 | 133 | 0003 + -1 | 213 | 0003 | -1 | 143 | 0003 + -1 | 213 | 0003 | -1 | 203 | 0003 + -1 | 213 | 0003 | -1 | 213 | 0003 + -1 | 213 | 0003 | -1 | 223 | 0003 + -1 | 223 | 0003 | -1 | 133 | 0003 + -1 | 223 | 0003 | -1 | 143 | 0003 + -1 | 223 | 0003 | -1 | 203 | 0003 + -1 | 223 | 0003 | -1 | 213 | 0003 + -1 | 223 | 0003 | -1 | 223 | 0003 + 1 | 133 | 0003 | 1 | 133 | 0003 + 1 | 133 | 0003 | 1 | 143 | 0003 + 1 | 133 | 0003 | 1 | 203 | 0003 + 1 | 133 | 0003 | 1 | 213 | 0003 + 1 | 133 | 0003 | 1 | 223 | 0003 + 1 | 143 | 0003 | 1 | 133 | 0003 + 1 | 143 | 0003 | 1 | 143 | 0003 + 1 | 143 | 0003 | 1 | 203 | 0003 + 1 | 143 | 0003 | 1 | 213 | 0003 + 1 | 143 | 0003 | 1 | 223 | 0003 + 1 | 153 | 0003 | 1 | 133 | 0003 + 1 | 153 | 0003 | 1 | 143 | 0003 + 1 | 153 | 0003 | 1 | 203 | 0003 + 1 | 153 | 0003 | 1 | 213 | 0003 + 1 | 153 | 0003 | 1 | 223 | 0003 + 1 | 163 | 0003 | 1 | 133 | 0003 + 1 | 163 | 0003 | 1 | 143 | 0003 + 1 | 163 | 0003 | 1 | 203 | 0003 + 1 | 163 | 0003 | 1 | 213 | 0003 + 1 | 163 | 0003 | 1 | 223 | 0003 + 1 | 173 | 0003 | 1 | 133 | 0003 + 1 | 173 | 0003 | 1 | 143 | 0003 + 1 | 173 | 0003 | 1 | 203 | 0003 + 1 | 173 | 0003 | 1 | 213 | 0003 + 1 | 173 | 0003 | 1 | 223 | 0003 + 1 | 183 | 0003 | 1 | 133 | 0003 + 1 | 183 | 0003 | 1 | 143 | 0003 + 1 | 183 | 0003 | 1 | 203 | 0003 + 1 | 183 | 0003 | 1 | 213 | 0003 + 1 | 183 | 0003 | 1 | 223 | 0003 + 1 | 193 | 0003 | 1 | 133 | 0003 + 1 | 193 | 0003 | 1 | 143 | 0003 + 1 | 193 | 0003 | 1 | 203 | 0003 + 1 | 193 | 0003 | 1 | 213 | 0003 + 1 | 193 | 0003 | 1 | 223 | 0003 + 1 | 203 | 0003 | 1 | 133 | 0003 + 1 | 203 | 0003 | 1 | 143 | 0003 + 1 | 203 | 0003 | 1 | 203 | 0003 + 1 | 203 | 0003 | 1 | 213 | 0003 + 1 | 203 | 0003 | 1 | 223 | 0003 + 1 | 213 | 0003 | 1 | 133 | 0003 + 1 | 213 | 0003 | 1 | 143 | 0003 + 1 | 213 | 0003 | 1 | 203 | 0003 + 1 | 213 | 0003 | 1 | 213 | 0003 + 1 | 213 | 0003 | 1 | 223 | 0003 + 1 | 223 | 0003 | 1 | 133 | 0003 + 1 | 223 | 0003 | 1 | 143 | 0003 + 1 | 223 | 0003 | 1 | 203 | 0003 + 1 | 223 | 0003 | 1 | 213 | 0003 + 1 | 223 | 0003 | 1 | 223 | 0003 +(100 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 > 300 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: ((t2_1.a = t1_1.a) AND (t2_1.b = t1_1.b) AND (t2_1.c = t1_1.c)) + -> Seq Scan on beta_neg_p3 t2_1 + -> Hash + -> Seq Scan on alpha_neg_p3 t1_1 + Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[]))) + -> Hash Join + Hash Cond: ((t2_2.a = t1_2.a) AND (t2_2.b = t1_2.b) AND (t2_2.c = t1_2.c)) + -> Seq Scan on beta_pos_p2 t2_2 + -> Hash + -> Seq Scan on alpha_pos_p2 t1_2 + Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[]))) + -> Hash Join + Hash Cond: ((t2_3.a = t1_3.a) AND (t2_3.b = t1_3.b) AND (t2_3.c = t1_3.c)) + -> Seq Scan on beta_pos_p3 t2_3 + -> Hash + -> Seq Scan on alpha_pos_p3 t1_3 + Filter: ((b > 300) AND (c = ANY ('{0004,0009}'::text[]))) +(21 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 < 300 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 | 114 | 0004 | -1 | 114 | 0004 + -1 | 119 | 0009 | -1 | 119 | 0009 + -1 | 124 | 0004 | -1 | 124 | 0004 + -1 | 129 | 0009 | -1 | 129 | 0009 + -1 | 134 | 0004 | -1 | 134 | 0004 + -1 | 139 | 0009 | -1 | 139 | 0009 + -1 | 144 | 0004 | -1 | 144 | 0004 + -1 | 149 | 0009 | -1 | 149 | 0009 + -1 | 204 | 0004 | -1 | 204 | 0004 + -1 | 209 | 0009 | -1 | 209 | 0009 + -1 | 214 | 0004 | -1 | 214 | 0004 + -1 | 219 | 0009 | -1 | 219 | 0009 + -1 | 224 | 0004 | -1 | 224 | 0004 + -1 | 229 | 0009 | -1 | 229 | 0009 + -1 | 234 | 0004 | -1 | 234 | 0004 + -1 | 239 | 0009 | -1 | 239 | 0009 + -1 | 244 | 0004 | -1 | 244 | 0004 + -1 | 249 | 0009 | -1 | 249 | 0009 + -1 | 254 | 0004 | -1 | 254 | 0004 + -1 | 259 | 0009 | -1 | 259 | 0009 + -1 | 264 | 0004 | -1 | 264 | 0004 + -1 | 269 | 0009 | -1 | 269 | 0009 + -1 | 274 | 0004 | -1 | 274 | 0004 + -1 | 279 | 0009 | -1 | 279 | 0009 + -1 | 284 | 0004 | -1 | 284 | 0004 + -1 | 289 | 0009 | -1 | 289 | 0009 + -1 | 294 | 0004 | -1 | 294 | 0004 + -1 | 299 | 0009 | -1 | 299 | 0009 + 1 | 104 | 0004 | 1 | 104 | 0004 + 1 | 109 | 0009 | 1 | 109 | 0009 + 1 | 114 | 0004 | 1 | 114 | 0004 + 1 | 119 | 0009 | 1 | 119 | 0009 + 1 | 124 | 0004 | 1 | 124 | 0004 + 1 | 129 | 0009 | 1 | 129 | 0009 + 1 | 134 | 0004 | 1 | 134 | 0004 + 1 | 139 | 0009 | 1 | 139 | 0009 + 1 | 144 | 0004 | 1 | 144 | 0004 + 1 | 149 | 0009 | 1 | 149 | 0009 + 1 | 204 | 0004 | 1 | 204 | 0004 + 1 | 209 | 0009 | 1 | 209 | 0009 + 1 | 214 | 0004 | 1 | 214 | 0004 + 1 | 219 | 0009 | 1 | 219 | 0009 + 1 | 224 | 0004 | 1 | 224 | 0004 + 1 | 229 | 0009 | 1 | 229 | 0009 + 1 | 234 | 0004 | 1 | 234 | 0004 + 1 | 239 | 0009 | 1 | 239 | 0009 + 1 | 244 | 0004 | 1 | 244 | 0004 + 1 | 249 | 0009 | 1 | 249 | 0009 + 1 | 254 | 0004 | 1 | 254 | 0004 + 1 | 259 | 0009 | 1 | 259 | 0009 + 1 | 264 | 0004 | 1 | 264 | 0004 + 1 | 269 | 0009 | 1 | 269 | 0009 + 1 | 274 | 0004 | 1 | 274 | 0004 + 1 | 279 | 0009 | 1 | 279 | 0009 + 1 | 284 | 0004 | 1 | 284 | 0004 + 1 | 289 | 0009 | 1 | 289 | 0009 + 1 | 294 | 0004 | 1 | 294 | 0004 + 1 | 299 | 0009 | 1 | 299 | 0009 +(60 rows) diff --git a/src/test/regress/sql/partition_join.sql b/src/test/regress/sql/partition_join.sql index 575ba7b8d4..31dcd6e77c 100644 --- a/src/test/regress/sql/partition_join.sql +++ b/src/test/regress/sql/partition_join.sql @@ -445,11 +445,643 @@ 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_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_ad_a_idx on prt1_ad (a); +INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_ad; + +CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (150); +CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt2_ad_p3 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (500); +CREATE INDEX prt2_ad_b_idx on prt2_ad (b); +INSERT INTO prt2_ad_p1 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 149) i; +INSERT INTO prt2_ad_p2 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(200, 299) i; +INSERT INTO prt2_ad_p3 SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_ad; + +-- 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_ad t1 INNER JOIN prt2_ad 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_ad t1 INNER JOIN prt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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 + +-- Add an extra partition to prt2_ad +CREATE TABLE prt2_ad_extra PARTITION OF prt2_ad FOR VALUES FROM (500) TO (MAXVALUE); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(500, 599) i; +ANALYZE prt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 INNER JOIN prt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0 ORDER BY t1.a; +SELECT t1.* FROM prt1_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad t1 LEFT JOIN prt1_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; +SELECT t1.* FROM prt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + +-- 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt1_ad t2 WHERE t1.b = t2.a) AND t1.a = 0; + +-- 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_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c; +SELECT t1.a, t2.a, t3.c FROM prt1_ad t1 RIGHT JOIN prt2_ad t2 ON (t1.a = t2.b) INNER JOIN prt1_ad t3 ON (t2.b = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.a, t3.c; + +DROP TABLE prt2_ad_extra; + +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in those cases + +-- Split prt2_ad_p3 into two partitions so that prt1_ad_p3 matches both +-- partitions +ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3; +CREATE TABLE prt2_ad_p3_350_375 PARTITION OF prt2_ad FOR VALUES FROM (350) TO (375); +CREATE TABLE prt2_ad_p3_375_500 PARTITION OF prt2_ad FOR VALUES FROM (375) TO (500); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(350, 499) i; +ANALYZE prt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM prt2_ad 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_ad t1 LEFT JOIN prt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM prt2_ad t2 WHERE t1.a = t2.b) AND t1.b = 0; + +-- full join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM (SELECT 175 phv, * FROM prt1_ad WHERE prt1_ad.b = 0) t1 FULL JOIN (SELECT 425 phv, * FROM prt2_ad WHERE prt2_ad.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_ad_p3_350_375; +DROP TABLE prt2_ad_p3_375_500; + +-- Test default partitions + +-- Change prt1_ad_p1 to the default partition +ALTER TABLE prt1_ad DETACH PARTITION prt1_ad_p1; +ALTER TABLE prt1_ad ATTACH PARTITION prt1_ad_p1 DEFAULT; +ANALYZE prt1_ad; + +CREATE TABLE prt2_ad_p3_300_400 PARTITION OF prt2_ad FOR VALUES FROM (300) TO (400); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(300, 399) i; +ANALYZE prt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt2_ad_p3_300_400; + +-- Restore prt2_ad_p3 +ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 FOR VALUES FROM (350) TO (500); +ANALYZE prt2_ad; + +-- Partitioned join can't be applied because the default partition of prt1_ad +-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +-- Change prt2_ad_p3 to the default partition +ALTER TABLE prt2_ad DETACH PARTITION prt2_ad_p3; +ALTER TABLE prt2_ad ATTACH PARTITION prt2_ad_p3 DEFAULT; +ANALYZE prt2_ad; + +-- Partitioned join can't be applied because the default partition of prt1_ad +-- prt1_ad_p1 matches prt2_ad_p1 and prt2_ad_p3 +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt1_ad_p3; +ANALYZE prt1_ad; + +DROP TABLE prt2_ad_p3; +ANALYZE prt2_ad; + +CREATE TABLE prt3_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt3_ad_p1 PARTITION OF prt3_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt3_ad_p2 PARTITION OF prt3_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt3_ad_a_idx on prt3_ad (a); +INSERT INTO prt3_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(200, 399) i; +ANALYZE prt3_ad; + +-- 3-way join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c, t3.a, t3.c FROM prt1_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad 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_ad t1 LEFT JOIN prt2_ad t2 ON (t1.a = t2.b) LEFT JOIN prt3_ad t3 ON (t1.a = t3.a) WHERE t1.b = 0 ORDER BY t1.a, t2.b, t3.a; + +DROP TABLE prt1_ad; +DROP TABLE prt2_ad; +DROP TABLE prt3_ad; + +-- Test interaction of partitioned join with partition pruning + +CREATE TABLE prt1_ad (a int, b int, c varchar) PARTITION BY RANGE (a); +CREATE TABLE prt1_ad_p1 PARTITION OF prt1_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt1_ad_p2 PARTITION OF prt1_ad FOR VALUES FROM (200) TO (300); +CREATE TABLE prt1_ad_p3 PARTITION OF prt1_ad FOR VALUES FROM (300) TO (400); +CREATE INDEX prt1_ad_a_idx on prt1_ad (a); +INSERT INTO prt1_ad SELECT i, i % 25, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt1_ad; + +CREATE TABLE prt2_ad (a int, b int, c varchar) PARTITION BY RANGE (b); +CREATE TABLE prt2_ad_p1 PARTITION OF prt2_ad FOR VALUES FROM (100) TO (200); +CREATE TABLE prt2_ad_p2 PARTITION OF prt2_ad FOR VALUES FROM (200) TO (400); +CREATE INDEX prt2_ad_b_idx on prt2_ad (b); +INSERT INTO prt2_ad SELECT i % 25, i, to_char(i, 'FM0000') FROM generate_series(100, 399) i; +ANALYZE prt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 INNER JOIN prt2_ad t2 ON (t1.a = t2.b) WHERE t1.a < 300 AND t1.b = 0 ORDER BY t1.a, t2.b; + +DROP TABLE prt1_ad_p3; +CREATE TABLE prt1_ad_default PARTITION OF prt1_ad DEFAULT; +ANALYZE prt1_ad; + +CREATE TABLE prt2_ad_default PARTITION OF prt2_ad DEFAULT; +ANALYZE prt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.b, t2.c FROM prt1_ad t1 INNER JOIN prt2_ad 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_ad t1 INNER JOIN prt2_ad 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_ad; +DROP TABLE prt2_ad; + +-- Tests for list-partitioned tables +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001', '0003'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0008', '0009'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 2, 5, 7); +ANALYZE plt1_ad; + +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002', '0003'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN ('0007', '0009'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 NOT IN (0, 1, 5, 8); +ANALYZE plt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad t1 FULL JOIN plt2_ad 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 + +-- Add an extra partition to plt2_ad +CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN ('0000'); +INSERT INTO plt2_ad_extra VALUES (0, 0, '0000'); +ANALYZE plt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt1_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt1_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_extra; + +-- Test cases where a partition on one side matches multiple partitions on +-- the other side; we currently can't do partitioned join in those cases + +-- Split plt2_ad_p2 into two partitions so that prt1_ad_p2 matches both +-- partitions +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2; +CREATE TABLE plt2_ad_p2_0004 PARTITION OF plt2_ad FOR VALUES IN ('0004'); +CREATE TABLE plt2_ad_p2_0006 PARTITION OF plt2_ad FOR VALUES IN ('0006'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6); +ANALYZE plt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_p2_0004; +DROP TABLE plt2_ad_p2_0006; + +-- Restore plt2_ad_p2 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_ad; + +-- Test NULL partitions + +-- Change plt1_ad_p1 to the NULL partition +ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1; +CREATE TABLE plt1_ad_p1_null PARTITION OF plt1_ad FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt1_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; + +-- Change plt2_ad_p3 to the NULL partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p3; +CREATE TABLE plt2_ad_p3_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0007', '0009'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (7, 9); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; + +-- inner join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad 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_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad t2 WHERE t1.a = t2.a AND t1.c = t2.c) AND t1.b < 10 ORDER BY t1.a; +SELECT t1.* FROM plt1_ad t1 WHERE NOT EXISTS (SELECT 1 FROM plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_p1_null; +DROP TABLE plt2_ad_p3_null; + +-- Restore plt1_ad_p1 +ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 FOR VALUES IN ('0001', '0003'); + +-- Add to plt1_ad the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt1_ad_extra PARTITION OF plt1_ad FOR VALUES IN (NULL); +INSERT INTO plt1_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; + +-- Restore plt2_ad_p3 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p3 FOR VALUES IN ('0007', '0009'); +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Partitioned join can't be applied because there isn't any partition on the +-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Partitioned join can't be applied because there isn't any partition on the +-- plt2_ad side that matches the NULL partition of plt1_ad plt1_ad_extra +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad the extra NULL partition containing only NULL values as the +-- key values +CREATE TABLE plt2_ad_extra PARTITION OF plt2_ad FOR VALUES IN (NULL); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 FULL JOIN plt2_ad 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_ad t1 FULL JOIN plt2_ad 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_ad_extra; +DROP TABLE plt2_ad_extra; + +-- Test default partitions + +-- Change plt1_ad_p1 to the default partition +ALTER TABLE plt1_ad DETACH PARTITION plt1_ad_p1; +ALTER TABLE plt1_ad ATTACH PARTITION plt1_ad_p1 DEFAULT; +DROP TABLE plt1_ad_p3; +ANALYZE plt1_ad; + +DROP TABLE plt2_ad_p3; +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Change plt2_ad_p2 to contain '0005' in addition to '0004' and '0006' as +-- the key values +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2; +CREATE TABLE plt2_ad_p2_ext PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005', '0006'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 5, 6); +ANALYZE plt2_ad; + +-- Partitioned join can't be applied because the default partition of plt1_ad +-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Change plt2_ad_p2_ext to the default partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p2_ext; +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2_ext DEFAULT; +ANALYZE plt2_ad; + +-- Partitioned join can't be applied because the default partition of plt1_ad +-- plt1_ad_p1 matches plt2_ad_p1 and plt2_ad_p2_ext +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt2_ad_p2_ext; + +-- Restore plt2_ad_p2 +ALTER TABLE plt2_ad ATTACH PARTITION plt2_ad_p2 FOR VALUES IN ('0004', '0006'); +ANALYZE plt2_ad; + +CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0004', '0006'); +CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0007', '0009'); +INSERT INTO plt3_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (4, 6, 7, 9); +ANALYZE plt3_ad; + +-- 3-way join +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad 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_ad t1 LEFT JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) LEFT JOIN plt3_ad 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 + +-- Change plt2_ad_p1 to the NULL partition +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1; +CREATE TABLE plt2_ad_p1_null PARTITION OF plt2_ad FOR VALUES IN (NULL, '0001', '0003'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 10, 'FM0000') FROM generate_series(1, 299) i WHERE i % 10 IN (1, 3); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +-- Change the NULL partition to contain only NULL values as the key values +ALTER TABLE plt2_ad DETACH PARTITION plt2_ad_p1_null; +CREATE TABLE plt2_ad_p1_nullonly PARTITION OF plt2_ad FOR VALUES IN (NULL); +INSERT INTO plt2_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad 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_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.b < 10 ORDER BY t1.a; + +DROP TABLE plt2_ad_p1_nullonly; +DROP TABLE plt2_ad_p1_null; +DROP TABLE plt2_ad_p1; + +DROP TABLE plt1_ad; +DROP TABLE plt2_ad; +DROP TABLE plt3_ad; + + +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0000', '0001', '0002'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt1_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i; +ANALYZE plt1_ad; + +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0002'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt2_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (2, 3, 4); +ANALYZE plt2_ad; + +CREATE TABLE plt3_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt3_ad_p1 PARTITION OF plt3_ad FOR VALUES IN ('0001'); +CREATE TABLE plt3_ad_p2 PARTITION OF plt3_ad FOR VALUES IN ('0003', '0004'); +INSERT INTO plt3_ad SELECT i, i, to_char(i % 5, 'FM0000') FROM generate_series(0, 24) i WHERE i % 5 IN (1, 3, 4); +ANALYZE plt3_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c, t3.a, t3.c FROM (plt1_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad 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_ad t1 LEFT JOIN plt2_ad t2 ON (t1.c = t2.c)) FULL JOIN plt3_ad 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_ad; +DROP TABLE plt2_ad; +DROP TABLE plt3_ad; + +-- Test interaction of partitioned join with partition pruning + +CREATE TABLE plt1_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt1_ad_p1 PARTITION OF plt1_ad FOR VALUES IN ('0001'); +CREATE TABLE plt1_ad_p2 PARTITION OF plt1_ad FOR VALUES IN ('0002'); +CREATE TABLE plt1_ad_p3 PARTITION OF plt1_ad FOR VALUES IN ('0003'); +CREATE TABLE plt1_ad_p4 PARTITION OF plt1_ad FOR VALUES IN (NULL, '0004', '0005'); +INSERT INTO plt1_ad 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_ad VALUES (-1, -1, NULL); +ANALYZE plt1_ad; + +CREATE TABLE plt2_ad (a int, b int, c text) PARTITION BY LIST (c); +CREATE TABLE plt2_ad_p1 PARTITION OF plt2_ad FOR VALUES IN ('0001', '0002'); +CREATE TABLE plt2_ad_p2 PARTITION OF plt2_ad FOR VALUES IN ('0003'); +CREATE TABLE plt2_ad_p3 PARTITION OF plt2_ad FOR VALUES IN (NULL); +CREATE TABLE plt2_ad_p4 PARTITION OF plt2_ad FOR VALUES IN ('0004', '0005'); +INSERT INTO plt2_ad 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_ad VALUES (-1, -1, NULL); +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad_default PARTITION OF plt1_ad DEFAULT; +ANALYZE plt1_ad; + +CREATE TABLE plt2_ad_default PARTITION OF plt2_ad DEFAULT; +ANALYZE plt2_ad; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a; +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 INNER JOIN plt2_ad t2 ON (t1.a = t2.a AND t1.c = t2.c) WHERE t1.c IN ('0003', '0004', '0005') AND t1.b < 10 AND t2.c IS NOT NULL ORDER BY t1.a; + +EXPLAIN (COSTS OFF) +SELECT t1.a, t1.c, t2.a, t2.c FROM plt1_ad t1 LEFT JOIN plt2_ad 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_ad t1 LEFT JOIN plt2_ad 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_ad; +DROP TABLE plt2_ad; + +-- 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 >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, 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 >= 125 AND t1.b < 225 AND t2.b >= 125 AND t2.b < 225 AND t1.c IN ('0003') ORDER BY t1.a, t1.b, t2.a, 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 > 300 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 < 300 AND t1.c IN ('0004', '0009') ORDER BY t1.a, t1.b;