Commit 0568e7a2 authored by Tom Lane's avatar Tom Lane

Cosmetic improvements for code related to partitionwise join.

Move have_partkey_equi_join and match_expr_to_partition_keys to
relnode.c, since they're used only there.  Refactor
build_joinrel_partition_info to split out the code that fills the
joinrel's partition key lists; this doesn't have any non-cosmetic
impact, but it seems like a useful separation of concerns.
Improve assorted nearby comments.

Amit Langote, with a little further editorialization by me

Discussion: https://postgr.es/m/CA+HiwqG2WVUGmLJqtR0tPFhniO=H=9qQ+Z3L_ZC+Y3-EVQHFGg@mail.gmail.com
parent 21dc4884
...@@ -20,7 +20,6 @@ ...@@ -20,7 +20,6 @@
#include "optimizer/pathnode.h" #include "optimizer/pathnode.h"
#include "optimizer/paths.h" #include "optimizer/paths.h"
#include "partitioning/partbounds.h" #include "partitioning/partbounds.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h" #include "utils/memutils.h"
...@@ -46,8 +45,6 @@ static void try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1, ...@@ -46,8 +45,6 @@ static void try_partitionwise_join(PlannerInfo *root, RelOptInfo *rel1,
static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root, static SpecialJoinInfo *build_child_join_sjinfo(PlannerInfo *root,
SpecialJoinInfo *parent_sjinfo, SpecialJoinInfo *parent_sjinfo,
Relids left_relids, Relids right_relids); Relids left_relids, Relids right_relids);
static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
bool strict_op);
/* /*
...@@ -1573,168 +1570,3 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo, ...@@ -1573,168 +1570,3 @@ build_child_join_sjinfo(PlannerInfo *root, SpecialJoinInfo *parent_sjinfo,
return sjinfo; return sjinfo;
} }
/*
* Returns true if there exists an equi-join condition for each pair of
* partition keys from given relations being joined.
*/
bool
have_partkey_equi_join(RelOptInfo *joinrel,
RelOptInfo *rel1, RelOptInfo *rel2,
JoinType jointype, List *restrictlist)
{
PartitionScheme part_scheme = rel1->part_scheme;
ListCell *lc;
int cnt_pks;
bool pk_has_clause[PARTITION_MAX_KEYS];
bool strict_op;
/*
* This function should be called when the joining relations have same
* partitioning scheme.
*/
Assert(rel1->part_scheme == rel2->part_scheme);
Assert(part_scheme);
memset(pk_has_clause, 0, sizeof(pk_has_clause));
foreach(lc, restrictlist)
{
RestrictInfo *rinfo = lfirst_node(RestrictInfo, lc);
OpExpr *opexpr;
Expr *expr1;
Expr *expr2;
int ipk1;
int ipk2;
/* If processing an outer join, only use its own join clauses. */
if (IS_OUTER_JOIN(jointype) &&
RINFO_IS_PUSHED_DOWN(rinfo, joinrel->relids))
continue;
/* Skip clauses which can not be used for a join. */
if (!rinfo->can_join)
continue;
/* Skip clauses which are not equality conditions. */
if (!rinfo->mergeopfamilies && !OidIsValid(rinfo->hashjoinoperator))
continue;
opexpr = castNode(OpExpr, rinfo->clause);
/*
* The equi-join between partition keys is strict if equi-join between
* at least one partition key is using a strict operator. See
* explanation about outer join reordering identity 3 in
* optimizer/README
*/
strict_op = op_strict(opexpr->opno);
/* Match the operands to the relation. */
if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
bms_is_subset(rinfo->right_relids, rel2->relids))
{
expr1 = linitial(opexpr->args);
expr2 = lsecond(opexpr->args);
}
else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
bms_is_subset(rinfo->right_relids, rel1->relids))
{
expr1 = lsecond(opexpr->args);
expr2 = linitial(opexpr->args);
}
else
continue;
/*
* Only clauses referencing the partition keys are useful for
* partitionwise join.
*/
ipk1 = match_expr_to_partition_keys(expr1, rel1, strict_op);
if (ipk1 < 0)
continue;
ipk2 = match_expr_to_partition_keys(expr2, rel2, strict_op);
if (ipk2 < 0)
continue;
/*
* If the clause refers to keys at different ordinal positions, it can
* not be used for partitionwise join.
*/
if (ipk1 != ipk2)
continue;
/*
* The clause allows partitionwise join if only it uses the same
* operator family as that specified by the partition key.
*/
if (rel1->part_scheme->strategy == PARTITION_STRATEGY_HASH)
{
if (!op_in_opfamily(rinfo->hashjoinoperator,
part_scheme->partopfamily[ipk1]))
continue;
}
else if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;
/* Mark the partition key as having an equi-join clause. */
pk_has_clause[ipk1] = true;
}
/* Check whether every partition key has an equi-join condition. */
for (cnt_pks = 0; cnt_pks < part_scheme->partnatts; cnt_pks++)
{
if (!pk_has_clause[cnt_pks])
return false;
}
return true;
}
/*
* Find the partition key from the given relation matching the given
* expression. If found, return the index of the partition key, else return -1.
*/
static int
match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
{
int cnt;
/* This function should be called only for partitioned relations. */
Assert(rel->part_scheme);
/* Remove any relabel decorations. */
while (IsA(expr, RelabelType))
expr = (Expr *) (castNode(RelabelType, expr))->arg;
for (cnt = 0; cnt < rel->part_scheme->partnatts; cnt++)
{
ListCell *lc;
Assert(rel->partexprs);
foreach(lc, rel->partexprs[cnt])
{
if (equal(lfirst(lc), expr))
return cnt;
}
if (!strict_op)
continue;
/*
* If it's a strict equi-join a NULL partition key on one side will
* not join a NULL partition key on the other side. So, rows with NULL
* partition key from a partition on one side can not join with those
* from a non-matching partition on the other side. So, search the
* nullable partition keys as well.
*/
Assert(rel->nullable_partexprs);
foreach(lc, rel->nullable_partexprs[cnt])
{
if (equal(lfirst(lc), expr))
return cnt;
}
}
return -1;
}
...@@ -2250,9 +2250,8 @@ find_partition_scheme(PlannerInfo *root, Relation relation) ...@@ -2250,9 +2250,8 @@ find_partition_scheme(PlannerInfo *root, Relation relation)
/* /*
* set_baserel_partition_key_exprs * set_baserel_partition_key_exprs
* *
* Builds partition key expressions for the given base relation and sets them * Builds partition key expressions for the given base relation and fills
* in given RelOptInfo. Any single column partition keys are converted to Var * rel->partexprs.
* nodes. All Var nodes are restamped with the relid of given relation.
*/ */
static void static void
set_baserel_partition_key_exprs(Relation relation, set_baserel_partition_key_exprs(Relation relation,
...@@ -2300,16 +2299,17 @@ set_baserel_partition_key_exprs(Relation relation, ...@@ -2300,16 +2299,17 @@ set_baserel_partition_key_exprs(Relation relation,
lc = lnext(partkey->partexprs, lc); lc = lnext(partkey->partexprs, lc);
} }
/* Base relations have a single expression per key. */
partexprs[cnt] = list_make1(partexpr); partexprs[cnt] = list_make1(partexpr);
} }
rel->partexprs = partexprs; rel->partexprs = partexprs;
/* /*
* A base relation can not have nullable partition key expressions. We * A base relation does not have nullable partition key expressions, since
* still allocate array of empty expressions lists to keep partition key * no outer join is involved. We still allocate an array of empty
* expression handling code simple. See build_joinrel_partition_info() and * expression lists to keep partition key expression handling code simple.
* match_expr_to_partition_keys(). * See build_joinrel_partition_info() and match_expr_to_partition_keys().
*/ */
rel->nullable_partexprs = (List **) palloc0(sizeof(List *) * partnatts); rel->nullable_partexprs = (List **) palloc0(sizeof(List *) * partnatts);
} }
......
...@@ -29,6 +29,7 @@ ...@@ -29,6 +29,7 @@
#include "optimizer/tlist.h" #include "optimizer/tlist.h"
#include "partitioning/partbounds.h" #include "partitioning/partbounds.h"
#include "utils/hsearch.h" #include "utils/hsearch.h"
#include "utils/lsyscache.h"
typedef struct JoinHashEntry typedef struct JoinHashEntry
...@@ -58,6 +59,14 @@ static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel); ...@@ -58,6 +59,14 @@ static void add_join_rel(PlannerInfo *root, RelOptInfo *joinrel);
static void build_joinrel_partition_info(RelOptInfo *joinrel, static void build_joinrel_partition_info(RelOptInfo *joinrel,
RelOptInfo *outer_rel, RelOptInfo *inner_rel, RelOptInfo *outer_rel, RelOptInfo *inner_rel,
List *restrictlist, JoinType jointype); List *restrictlist, JoinType jointype);
static bool have_partkey_equi_join(RelOptInfo *joinrel,
RelOptInfo *rel1, RelOptInfo *rel2,
JoinType jointype, List *restrictlist);
static int match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel,
bool strict_op);
static void set_joinrel_partition_key_exprs(RelOptInfo *joinrel,
RelOptInfo *outer_rel, RelOptInfo *inner_rel,
JoinType jointype);
static void build_child_join_reltarget(PlannerInfo *root, static void build_child_join_reltarget(PlannerInfo *root,
RelOptInfo *parentrel, RelOptInfo *parentrel,
RelOptInfo *childrel, RelOptInfo *childrel,
...@@ -1607,18 +1616,15 @@ find_param_path_info(RelOptInfo *rel, Relids required_outer) ...@@ -1607,18 +1616,15 @@ find_param_path_info(RelOptInfo *rel, Relids required_outer)
/* /*
* build_joinrel_partition_info * build_joinrel_partition_info
* If the two relations have same partitioning scheme, their join may be * Checks if the two relations being joined can use partitionwise join
* partitioned and will follow the same partitioning scheme as the joining * and if yes, initialize partitioning information of the resulting
* relations. Set the partition scheme and partition key expressions in * partitioned join relation.
* the join relation.
*/ */
static void static void
build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
RelOptInfo *inner_rel, List *restrictlist, RelOptInfo *inner_rel, List *restrictlist,
JoinType jointype) JoinType jointype)
{ {
int partnatts;
int cnt;
PartitionScheme part_scheme; PartitionScheme part_scheme;
/* Nothing to do if partitionwise join technique is disabled. */ /* Nothing to do if partitionwise join technique is disabled. */
...@@ -1672,8 +1678,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1672,8 +1678,7 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
/* /*
* This function will be called only once for each joinrel, hence it * This function will be called only once for each joinrel, hence it
* should not have partition scheme, partition bounds, partition key * should not have partitioning fields filled yet.
* expressions and array for storing child relations set.
*/ */
Assert(!joinrel->part_scheme && !joinrel->partexprs && Assert(!joinrel->part_scheme && !joinrel->partexprs &&
!joinrel->nullable_partexprs && !joinrel->part_rels && !joinrel->nullable_partexprs && !joinrel->part_rels &&
...@@ -1685,11 +1690,10 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1685,11 +1690,10 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
*/ */
joinrel->part_scheme = part_scheme; joinrel->part_scheme = part_scheme;
joinrel->boundinfo = outer_rel->boundinfo; 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->nparts = outer_rel->nparts;
set_joinrel_partition_key_exprs(joinrel, outer_rel, inner_rel, jointype);
/* part_rels[] will be filled later, but allocate it now */
joinrel->part_rels = joinrel->part_rels =
(RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts); (RelOptInfo **) palloc0(sizeof(RelOptInfo *) * joinrel->nparts);
...@@ -1699,34 +1703,205 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1699,34 +1703,205 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
Assert(outer_rel->consider_partitionwise_join); Assert(outer_rel->consider_partitionwise_join);
Assert(inner_rel->consider_partitionwise_join); Assert(inner_rel->consider_partitionwise_join);
joinrel->consider_partitionwise_join = true; joinrel->consider_partitionwise_join = true;
}
/*
* have_partkey_equi_join
*
* Returns true if there exist equi-join conditions involving pairs
* of matching partition keys of the relations being joined for all
* partition keys.
*/
static bool
have_partkey_equi_join(RelOptInfo *joinrel,
RelOptInfo *rel1, RelOptInfo *rel2,
JoinType jointype, List *restrictlist)
{
PartitionScheme part_scheme = rel1->part_scheme;
ListCell *lc;
int cnt_pks;
bool pk_has_clause[PARTITION_MAX_KEYS];
bool strict_op;
/* /*
* Construct partition keys for the join. * This function must only be called when the joined relations have same
* * partitioning scheme.
* An INNER join between two partitioned relations can be regarded as */
* partitioned by either key expression. For example, A INNER JOIN B ON Assert(rel1->part_scheme == rel2->part_scheme);
* A.a = B.b can be regarded as partitioned on A.a or on B.b; they are Assert(part_scheme);
* equivalent.
* memset(pk_has_clause, 0, sizeof(pk_has_clause));
* For a SEMI or ANTI join, the result can only be regarded as being foreach(lc, restrictlist)
* partitioned in the same manner as the outer side, since the inner {
* columns are not retained. RestrictInfo *rinfo = lfirst_node(RestrictInfo, lc);
* OpExpr *opexpr;
* An OUTER join like (A LEFT JOIN B ON A.a = B.b) may produce rows with Expr *expr1;
* B.b NULL. These rows may not fit the partitioning conditions imposed on Expr *expr2;
* B.b. Hence, strictly speaking, the join is not partitioned by B.b and int ipk1;
* thus partition keys of an OUTER join should include partition key int ipk2;
* expressions from the OUTER side only. However, because all
* commonly-used comparison operators are strict, the presence of nulls on /* If processing an outer join, only use its own join clauses. */
* the outer side doesn't cause any problem; they can't match anything at if (IS_OUTER_JOIN(jointype) &&
* future join levels anyway. Therefore, we track two sets of RINFO_IS_PUSHED_DOWN(rinfo, joinrel->relids))
* expressions: those that authentically partition the relation continue;
* (partexprs) and those that partition the relation with the exception
* that extra nulls may be present (nullable_partexprs). When the /* Skip clauses which can not be used for a join. */
* comparison operator is strict, the latter is just as good as the if (!rinfo->can_join)
* former. continue;
/* Skip clauses which are not equality conditions. */
if (!rinfo->mergeopfamilies && !OidIsValid(rinfo->hashjoinoperator))
continue;
/* Should be OK to assume it's an OpExpr. */
opexpr = castNode(OpExpr, rinfo->clause);
/* Match the operands to the relation. */
if (bms_is_subset(rinfo->left_relids, rel1->relids) &&
bms_is_subset(rinfo->right_relids, rel2->relids))
{
expr1 = linitial(opexpr->args);
expr2 = lsecond(opexpr->args);
}
else if (bms_is_subset(rinfo->left_relids, rel2->relids) &&
bms_is_subset(rinfo->right_relids, rel1->relids))
{
expr1 = lsecond(opexpr->args);
expr2 = linitial(opexpr->args);
}
else
continue;
/*
* Now we need to know whether the join operator is strict; see
* comments in pathnodes.h.
*/
strict_op = op_strict(opexpr->opno);
/*
* Only clauses referencing the partition keys are useful for
* partitionwise join.
*/
ipk1 = match_expr_to_partition_keys(expr1, rel1, strict_op);
if (ipk1 < 0)
continue;
ipk2 = match_expr_to_partition_keys(expr2, rel2, strict_op);
if (ipk2 < 0)
continue;
/*
* If the clause refers to keys at different ordinal positions, it can
* not be used for partitionwise join.
*/
if (ipk1 != ipk2)
continue;
/*
* The clause allows partitionwise join only if it uses the same
* operator family as that specified by the partition key.
*/
if (rel1->part_scheme->strategy == PARTITION_STRATEGY_HASH)
{
if (!OidIsValid(rinfo->hashjoinoperator) ||
!op_in_opfamily(rinfo->hashjoinoperator,
part_scheme->partopfamily[ipk1]))
continue;
}
else if (!list_member_oid(rinfo->mergeopfamilies,
part_scheme->partopfamily[ipk1]))
continue;
/* Mark the partition key as having an equi-join clause. */
pk_has_clause[ipk1] = true;
}
/* Check whether every partition key has an equi-join condition. */
for (cnt_pks = 0; cnt_pks < part_scheme->partnatts; cnt_pks++)
{
if (!pk_has_clause[cnt_pks])
return false;
}
return true;
}
/*
* match_expr_to_partition_keys
*
* Tries to match an expression to one of the nullable or non-nullable
* partition keys of "rel". Returns the matched key's ordinal position,
* or -1 if the expression could not be matched to any of the keys.
*
* strict_op must be true if the expression will be compared with the
* partition key using a strict operator. This allows us to consider
* nullable as well as nonnullable partition keys.
*/
static int
match_expr_to_partition_keys(Expr *expr, RelOptInfo *rel, bool strict_op)
{
int cnt;
/* This function should be called only for partitioned relations. */
Assert(rel->part_scheme);
Assert(rel->partexprs);
Assert(rel->nullable_partexprs);
/* Remove any relabel decorations. */
while (IsA(expr, RelabelType))
expr = (Expr *) (castNode(RelabelType, expr))->arg;
for (cnt = 0; cnt < rel->part_scheme->partnatts; cnt++)
{
ListCell *lc;
/* We can always match to the non-nullable partition keys. */
foreach(lc, rel->partexprs[cnt])
{
if (equal(lfirst(lc), expr))
return cnt;
}
if (!strict_op)
continue;
/*
* If it's a strict join operator then a NULL partition key on one
* side will not join to any partition key on the other side, and in
* particular such a row can't join to a row from a different
* partition on the other side. So, it's okay to search the nullable
* partition keys as well.
*/
foreach(lc, rel->nullable_partexprs[cnt])
{
if (equal(lfirst(lc), expr))
return cnt;
}
}
return -1;
}
/*
* set_joinrel_partition_key_exprs
* Initialize partition key expressions for a partitioned joinrel.
*/
static void
set_joinrel_partition_key_exprs(RelOptInfo *joinrel,
RelOptInfo *outer_rel, RelOptInfo *inner_rel,
JoinType jointype)
{
int partnatts = joinrel->part_scheme->partnatts;
joinrel->partexprs = (List **) palloc0(sizeof(List *) * partnatts);
joinrel->nullable_partexprs =
(List **) palloc0(sizeof(List *) * partnatts);
/*
* The joinrel's partition expressions are the same as those of the input
* rels, but we must properly classify them as nullable or not in the
* joinrel's output.
*/ */
for (cnt = 0; cnt < partnatts; cnt++) for (int cnt = 0; cnt < partnatts; cnt++)
{ {
/* mark these const to enforce that we copy them properly */ /* mark these const to enforce that we copy them properly */
const List *outer_expr = outer_rel->partexprs[cnt]; const List *outer_expr = outer_rel->partexprs[cnt];
...@@ -1738,18 +1913,41 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1738,18 +1913,41 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
switch (jointype) switch (jointype)
{ {
/*
* A join relation resulting from an INNER join may be
* regarded as partitioned by either of the inner and outer
* relation keys. For example, A INNER JOIN B ON A.a = B.b
* can be regarded as partitioned on either A.a or B.b. So we
* add both keys to the joinrel's partexpr lists. However,
* anything that was already nullable still has to be treated
* as nullable.
*/
case JOIN_INNER: case JOIN_INNER:
partexpr = list_concat_copy(outer_expr, inner_expr); partexpr = list_concat_copy(outer_expr, inner_expr);
nullable_partexpr = list_concat_copy(outer_null_expr, nullable_partexpr = list_concat_copy(outer_null_expr,
inner_null_expr); inner_null_expr);
break; break;
/*
* A join relation resulting from a SEMI or ANTI join may be
* regarded as partitioned by the outer relation keys. The
* inner relation's keys are no longer interesting; since they
* aren't visible in the join output, nothing could join to
* them.
*/
case JOIN_SEMI: case JOIN_SEMI:
case JOIN_ANTI: case JOIN_ANTI:
partexpr = list_copy(outer_expr); partexpr = list_copy(outer_expr);
nullable_partexpr = list_copy(outer_null_expr); nullable_partexpr = list_copy(outer_null_expr);
break; break;
/*
* A join relation resulting from a LEFT OUTER JOIN likewise
* may be regarded as partitioned on the (non-nullable) outer
* relation keys. The inner (nullable) relation keys are okay
* as partition keys for further joins as long as they involve
* strict join operators.
*/
case JOIN_LEFT: case JOIN_LEFT:
partexpr = list_copy(outer_expr); partexpr = list_copy(outer_expr);
nullable_partexpr = list_concat_copy(inner_expr, nullable_partexpr = list_concat_copy(inner_expr,
...@@ -1758,6 +1956,12 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1758,6 +1956,12 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
inner_null_expr); inner_null_expr);
break; break;
/*
* For FULL OUTER JOINs, both relations are nullable, so the
* resulting join relation may be regarded as partitioned on
* either of inner and outer relation keys, but only for joins
* that involve strict join operators.
*/
case JOIN_FULL: case JOIN_FULL:
nullable_partexpr = list_concat_copy(outer_expr, nullable_partexpr = list_concat_copy(outer_expr,
inner_expr); inner_expr);
...@@ -1769,7 +1973,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel, ...@@ -1769,7 +1973,6 @@ build_joinrel_partition_info(RelOptInfo *joinrel, RelOptInfo *outer_rel,
default: default:
elog(ERROR, "unrecognized join type: %d", (int) jointype); elog(ERROR, "unrecognized join type: %d", (int) jointype);
} }
joinrel->partexprs[cnt] = partexpr; joinrel->partexprs[cnt] = partexpr;
......
...@@ -574,6 +574,24 @@ typedef struct PartitionSchemeData *PartitionScheme; ...@@ -574,6 +574,24 @@ typedef struct PartitionSchemeData *PartitionScheme;
* we know we will need it at least once (to price the sequential scan) * we know we will need it at least once (to price the sequential scan)
* and may need it multiple times to price index scans. * and may need it multiple times to price index scans.
* *
* A join relation is considered to be partitioned if it is formed from a
* join of two relations that are partitioned, have matching partitioning
* schemes, and are joined on an equijoin of the partitioning columns.
* Under those conditions we can consider the join relation to be partitioned
* by either relation's partitioning keys, though some care is needed if
* either relation can be forced to null by outer-joining. For example, an
* outer join like (A LEFT JOIN B ON A.a = B.b) may produce rows with B.b
* NULL. These rows may not fit the partitioning conditions imposed on B.
* Hence, strictly speaking, the join is not partitioned by B.b and thus
* partition keys of an outer join should include partition key expressions
* from the non-nullable side only. However, if a subsequent join uses
* strict comparison operators (and all commonly-used equijoin operators are
* strict), the presence of nulls doesn't cause a problem: such rows couldn't
* match anything on the other side and thus they don't create a need to do
* any cross-partition sub-joins. Hence we can treat such values as still
* partitioning the join output for the purpose of additional partitionwise
* joining, so long as a strict join operator is used by the next join.
*
* If the relation is partitioned, these fields will be set: * If the relation is partitioned, these fields will be set:
* *
* part_scheme - Partitioning scheme of the relation * part_scheme - Partitioning scheme of the relation
...@@ -586,16 +604,15 @@ typedef struct PartitionSchemeData *PartitionScheme; ...@@ -586,16 +604,15 @@ typedef struct PartitionSchemeData *PartitionScheme;
* this relation that are partitioned tables * this relation that are partitioned tables
* themselves, in hierarchical order * themselves, in hierarchical order
* *
* Note: A base relation always has only one set of partition keys, but a join * The partexprs and nullable_partexprs arrays each contain
* relation may have as many sets of partition keys as the number of relations * part_scheme->partnatts elements. Each of the elements is a list of
* being joined. partexprs and nullable_partexprs are arrays containing * partition key expressions. For partitioned base relations, there is one
* part_scheme->partnatts elements each. Each of these elements is a list of * expression in each partexprs element, and nullable_partexprs is empty.
* partition key expressions. For a base relation each list in partexprs * For partitioned join relations, each base relation within the join
* contains only one expression and nullable_partexprs is not populated. For a * contributes one partition key expression per partitioning column;
* join relation, partexprs and nullable_partexprs contain partition key * that expression goes in the partexprs[i] list if the base relation
* expressions from non-nullable and nullable relations resp. Lists at any * is not nullable by this join or any lower outer join, or in the
* given position in those arrays together contain as many elements as the * nullable_partexprs[i] list if the base relation is nullable.
* number of joining relations.
*---------- *----------
*/ */
typedef enum RelOptKind typedef enum RelOptKind
...@@ -716,16 +733,16 @@ typedef struct RelOptInfo ...@@ -716,16 +733,16 @@ typedef struct RelOptInfo
Relids top_parent_relids; /* Relids of topmost parents (if "other" Relids top_parent_relids; /* Relids of topmost parents (if "other"
* rel) */ * rel) */
/* used for partitioned relations */ /* used for partitioned relations: */
PartitionScheme part_scheme; /* Partitioning scheme. */ PartitionScheme part_scheme; /* Partitioning scheme */
int nparts; /* number of partitions */ int nparts; /* Number of partitions */
struct PartitionBoundInfoData *boundinfo; /* Partition bounds */ struct PartitionBoundInfoData *boundinfo; /* Partition bounds */
List *partition_qual; /* partition constraint */ List *partition_qual; /* Partition constraint, if not the root */
struct RelOptInfo **part_rels; /* Array of RelOptInfos of partitions, struct RelOptInfo **part_rels; /* Array of RelOptInfos of partitions,
* stored in the same order of bounds */ * stored in the same order as bounds */
List **partexprs; /* Non-nullable partition key expressions. */ List **partexprs; /* Non-nullable partition key expressions */
List **nullable_partexprs; /* Nullable partition key expressions. */ List **nullable_partexprs; /* Nullable partition key expressions */
List *partitioned_child_rels; /* List of RT indexes. */ List *partitioned_child_rels; /* List of RT indexes */
} RelOptInfo; } RelOptInfo;
/* /*
......
...@@ -106,9 +106,6 @@ extern bool have_join_order_restriction(PlannerInfo *root, ...@@ -106,9 +106,6 @@ extern bool have_join_order_restriction(PlannerInfo *root,
extern bool have_dangerous_phv(PlannerInfo *root, extern bool have_dangerous_phv(PlannerInfo *root,
Relids outer_relids, Relids inner_params); Relids outer_relids, Relids inner_params);
extern void mark_dummy_rel(RelOptInfo *rel); extern void mark_dummy_rel(RelOptInfo *rel);
extern bool have_partkey_equi_join(RelOptInfo *joinrel,
RelOptInfo *rel1, RelOptInfo *rel2,
JoinType jointype, List *restrictlist);
/* /*
* equivclass.c * equivclass.c
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment