Commit 959d00e9 authored by Tom Lane's avatar Tom Lane

Use Append rather than MergeAppend for scanning ordered partitions.

If we need ordered output from a scan of a partitioned table, but
the ordering matches the partition ordering, then we don't need to
use a MergeAppend to combine the pre-ordered per-partition scan
results: a plain Append will produce the same results.  This
both saves useless comparison work inside the MergeAppend proper,
and allows us to start returning tuples after istarting up just
the first child node not all of them.

However, all is not peaches and cream, because if some of the
child nodes have high startup costs then there will be big
discontinuities in the tuples-returned-versus-elapsed-time curve.
The planner's cost model cannot handle that (yet, anyway).
If we model the Append's startup cost as being just the first
child's startup cost, we may drastically underestimate the cost
of fetching slightly more tuples than are available from the first
child.  Since we've had bad experiences with over-optimistic choices
of "fast start" plans for ORDER BY LIMIT queries, that seems scary.
As a klugy workaround, set the startup cost estimate for an ordered
Append to be the sum of its children's startup costs (as MergeAppend
would).  This doesn't really describe reality, but it's less likely
to cause a bad plan choice than an underestimated startup cost would.
In practice, the cases where we really care about this optimization
will have child plans that are IndexScans with zero startup cost,
so that the overly conservative estimate is still just zero.

David Rowley, reviewed by Julien Rouhaud and Antonin Houska

Discussion: https://postgr.es/m/CAKJS1f-hAqhPLRk_RaSFTgYxd=Tz5hA7kQ2h4-DhJufQk8TGuw@mail.gmail.com
parent 9f06d79e
......@@ -840,6 +840,19 @@ ExecSetTupleBound(int64 tuples_needed, PlanState *child_node)
sortState->bound = tuples_needed;
}
}
else if (IsA(child_node, AppendState))
{
/*
* If it is an Append, we can apply the bound to any nodes that are
* children of the Append, since the Append surely need read no more
* than that many tuples from any one input.
*/
AppendState *aState = (AppendState *) child_node;
int i;
for (i = 0; i < aState->as_nplans; i++)
ExecSetTupleBound(tuples_needed, aState->appendplans[i]);
}
else if (IsA(child_node, MergeAppendState))
{
/*
......
......@@ -1847,6 +1847,7 @@ _outAppendPath(StringInfo str, const AppendPath *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(subpaths);
WRITE_INT_FIELD(first_partial_path);
WRITE_FLOAT_FIELD(limit_tuples, "%.0f");
}
static void
......
This diff is collapsed.
......@@ -1878,27 +1878,83 @@ cost_append(AppendPath *apath)
apath->path.startup_cost = 0;
apath->path.total_cost = 0;
apath->path.rows = 0;
if (apath->subpaths == NIL)
return;
if (!apath->path.parallel_aware)
{
Path *subpath = (Path *) linitial(apath->subpaths);
List *pathkeys = apath->path.pathkeys;
/*
* Startup cost of non-parallel-aware Append is the startup cost of
* first subpath.
*/
apath->path.startup_cost = subpath->startup_cost;
if (pathkeys == NIL)
{
Path *subpath = (Path *) linitial(apath->subpaths);
/* Compute rows and costs as sums of subplan rows and costs. */
foreach(l, apath->subpaths)
/*
* For an unordered, non-parallel-aware Append we take the startup
* cost as the startup cost of the first subpath.
*/
apath->path.startup_cost = subpath->startup_cost;
/* Compute rows and costs as sums of subplan rows and costs. */
foreach(l, apath->subpaths)
{
Path *subpath = (Path *) lfirst(l);
apath->path.rows += subpath->rows;
apath->path.total_cost += subpath->total_cost;
}
}
else
{
Path *subpath = (Path *) lfirst(l);
/*
* For an ordered, non-parallel-aware Append we take the startup
* cost as the sum of the subpath startup costs. This ensures
* that we don't underestimate the startup cost when a query's
* LIMIT is such that several of the children have to be run to
* satisfy it. This might be overkill --- another plausible hack
* would be to take the Append's startup cost as the maximum of
* the child startup costs. But we don't want to risk believing
* that an ORDER BY LIMIT query can be satisfied at small cost
* when the first child has small startup cost but later ones
* don't. (If we had the ability to deal with nonlinear cost
* interpolation for partial retrievals, we would not need to be
* so conservative about this.)
*
* This case is also different from the above in that we have to
* account for possibly injecting sorts into subpaths that aren't
* natively ordered.
*/
foreach(l, apath->subpaths)
{
Path *subpath = (Path *) lfirst(l);
Path sort_path; /* dummy for result of cost_sort */
apath->path.rows += subpath->rows;
apath->path.total_cost += subpath->total_cost;
if (!pathkeys_contained_in(pathkeys, subpath->pathkeys))
{
/*
* We'll need to insert a Sort node, so include costs for
* that. We can use the parent's LIMIT if any, since we
* certainly won't pull more than that many tuples from
* any child.
*/
cost_sort(&sort_path,
NULL, /* doesn't currently need root */
pathkeys,
subpath->total_cost,
subpath->rows,
subpath->pathtarget->width,
0.0,
work_mem,
apath->limit_tuples);
subpath = &sort_path;
}
apath->path.rows += subpath->rows;
apath->path.startup_cost += subpath->startup_cost;
apath->path.total_cost += subpath->total_cost;
}
}
}
else /* parallel-aware */
......@@ -1906,6 +1962,9 @@ cost_append(AppendPath *apath)
int i = 0;
double parallel_divisor = get_parallel_divisor(&apath->path);
/* Parallel-aware Append never produces ordered output. */
Assert(apath->path.pathkeys == NIL);
/* Calculate startup cost. */
foreach(l, apath->subpaths)
{
......
......@@ -1261,7 +1261,7 @@ mark_dummy_rel(RelOptInfo *rel)
/* Set up the dummy path */
add_path(rel, (Path *) create_append_path(NULL, rel, NIL, NIL,
rel->lateral_relids,
NIL, rel->lateral_relids,
0, false, NIL, -1));
/* Set or update cheapest_total_path and related fields */
......
......@@ -18,16 +18,21 @@
#include "postgres.h"
#include "access/stratnum.h"
#include "catalog/pg_opfamily.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
#include "nodes/plannodes.h"
#include "optimizer/optimizer.h"
#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "partitioning/partbounds.h"
#include "utils/lsyscache.h"
static bool pathkey_is_redundant(PathKey *new_pathkey, List *pathkeys);
static bool matches_boolean_partition_clause(RestrictInfo *rinfo,
RelOptInfo *partrel,
int partkeycol);
static bool right_merge_direction(PlannerInfo *root, PathKey *pathkey);
......@@ -546,6 +551,165 @@ build_index_pathkeys(PlannerInfo *root,
return retval;
}
/*
* partkey_is_bool_constant_for_query
*
* If a partition key column is constrained to have a constant value by the
* query's WHERE conditions, then it's irrelevant for sort-order
* considerations. Usually that means we have a restriction clause
* WHERE partkeycol = constant, which gets turned into an EquivalenceClass
* containing a constant, which is recognized as redundant by
* build_partition_pathkeys(). But if the partition key column is a
* boolean variable (or expression), then we are not going to see such a
* WHERE clause, because expression preprocessing will have simplified it
* to "WHERE partkeycol" or "WHERE NOT partkeycol". So we are not going
* to have a matching EquivalenceClass (unless the query also contains
* "ORDER BY partkeycol"). To allow such cases to work the same as they would
* for non-boolean values, this function is provided to detect whether the
* specified partition key column matches a boolean restriction clause.
*/
static bool
partkey_is_bool_constant_for_query(RelOptInfo *partrel, int partkeycol)
{
PartitionScheme partscheme = partrel->part_scheme;
ListCell *lc;
/* If the partkey isn't boolean, we can't possibly get a match */
if (!IsBooleanOpfamily(partscheme->partopfamily[partkeycol]))
return false;
/* Check each restriction clause for the partitioned rel */
foreach(lc, partrel->baserestrictinfo)
{
RestrictInfo *rinfo = (RestrictInfo *) lfirst(lc);
/* Ignore pseudoconstant quals, they won't match */
if (rinfo->pseudoconstant)
continue;
/* See if we can match the clause's expression to the partkey column */
if (matches_boolean_partition_clause(rinfo, partrel, partkeycol))
return true;
}
return false;
}
/*
* matches_boolean_partition_clause
* Determine if the boolean clause described by rinfo matches
* partrel's partkeycol-th partition key column.
*
* "Matches" can be either an exact match (equivalent to partkey = true),
* or a NOT above an exact match (equivalent to partkey = false).
*/
static bool
matches_boolean_partition_clause(RestrictInfo *rinfo,
RelOptInfo *partrel, int partkeycol)
{
Node *clause = (Node *) rinfo->clause;
Node *partexpr = (Node *) linitial(partrel->partexprs[partkeycol]);
/* Direct match? */
if (equal(partexpr, clause))
return true;
/* NOT clause? */
else if (is_notclause(clause))
{
Node *arg = (Node *) get_notclausearg((Expr *) clause);
if (equal(partexpr, arg))
return true;
}
return false;
}
/*
* build_partition_pathkeys
* Build a pathkeys list that describes the ordering induced by the
* partitions of partrel, under either forward or backward scan
* as per scandir.
*
* Caller must have checked that the partitions are properly ordered,
* as detected by partitions_are_ordered().
*
* Sets *partialkeys to true if pathkeys were only built for a prefix of the
* partition key, or false if the pathkeys include all columns of the
* partition key.
*/
List *
build_partition_pathkeys(PlannerInfo *root, RelOptInfo *partrel,
ScanDirection scandir, bool *partialkeys)
{
List *retval = NIL;
PartitionScheme partscheme = partrel->part_scheme;
int i;
Assert(partscheme != NULL);
Assert(partitions_are_ordered(partrel->boundinfo, partrel->nparts));
/* For now, we can only cope with baserels */
Assert(IS_SIMPLE_REL(partrel));
for (i = 0; i < partscheme->partnatts; i++)
{
PathKey *cpathkey;
Expr *keyCol = (Expr *) linitial(partrel->partexprs[i]);
/*
* Try to make a canonical pathkey for this partkey.
*
* We're considering a baserel scan, so nullable_relids should be
* NULL. Also, we assume the PartitionDesc lists any NULL partition
* last, so we treat the scan like a NULLS LAST index: we have
* nulls_first for backwards scan only.
*/
cpathkey = make_pathkey_from_sortinfo(root,
keyCol,
NULL,
partscheme->partopfamily[i],
partscheme->partopcintype[i],
partscheme->partcollation[i],
ScanDirectionIsBackward(scandir),
ScanDirectionIsBackward(scandir),
0,
partrel->relids,
false);
if (cpathkey)
{
/*
* We found the sort key in an EquivalenceClass, so it's relevant
* for this query. Add it to list, unless it's redundant.
*/
if (!pathkey_is_redundant(cpathkey, retval))
retval = lappend(retval, cpathkey);
}
else
{
/*
* Boolean partition keys might be redundant even if they do not
* appear in an EquivalenceClass, because of our special treatment
* of boolean equality conditions --- see the comment for
* partkey_is_bool_constant_for_query(). If that applies, we can
* continue to examine lower-order partition keys. Otherwise, the
* sort key is not an interesting sort order for this query, so we
* should stop considering partition columns; any lower-order sort
* keys won't be useful either.
*/
if (!partkey_is_bool_constant_for_query(partrel, i))
{
*partialkeys = true;
return retval;
}
}
}
*partialkeys = false;
return retval;
}
/*
* build_expression_pathkey
* Build a pathkeys list that describes an ordering by a single expression
......
......@@ -205,8 +205,6 @@ static NamedTuplestoreScan *make_namedtuplestorescan(List *qptlist, List *qpqual
Index scanrelid, char *enrname);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
static Append *make_append(List *appendplans, int first_partial_plan,
List *tlist, PartitionPruneInfo *partpruneinfo);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
......@@ -1060,10 +1058,16 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
{
Append *plan;
List *tlist = build_path_tlist(root, &best_path->path);
List *pathkeys = best_path->path.pathkeys;
List *subplans = NIL;
ListCell *subpaths;
RelOptInfo *rel = best_path->path.parent;
PartitionPruneInfo *partpruneinfo = NULL;
int nodenumsortkeys = 0;
AttrNumber *nodeSortColIdx = NULL;
Oid *nodeSortOperators = NULL;
Oid *nodeCollations = NULL;
bool *nodeNullsFirst = NULL;
/*
* The subpaths list could be empty, if every child was proven empty by
......@@ -1089,6 +1093,37 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
return plan;
}
/*
* Otherwise build an Append plan. Note that if there's just one child,
* the Append is pretty useless; but we wait till setrefs.c to get rid of
* it. Doing so here doesn't work because the varno of the child scan
* plan won't match the parent-rel Vars it'll be asked to emit.
*
* We don't have the actual creation of the Append node split out into a
* separate make_xxx function. This is because we want to run
* prepare_sort_from_pathkeys on it before we do so on the individual
* child plans, to make cross-checking the sort info easier.
*/
plan = makeNode(Append);
plan->plan.targetlist = tlist;
plan->plan.qual = NIL;
plan->plan.lefttree = NULL;
plan->plan.righttree = NULL;
if (pathkeys != NIL)
{
/* Compute sort column info, and adjust the Append's tlist as needed */
(void) prepare_sort_from_pathkeys((Plan *) plan, pathkeys,
best_path->path.parent->relids,
NULL,
true,
&nodenumsortkeys,
&nodeSortColIdx,
&nodeSortOperators,
&nodeCollations,
&nodeNullsFirst);
}
/* Build the plan for each child */
foreach(subpaths, best_path->subpaths)
{
......@@ -1098,6 +1133,63 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
/* Must insist that all children return the same tlist */
subplan = create_plan_recurse(root, subpath, CP_EXACT_TLIST);
/*
* For ordered Appends, we must insert a Sort node if subplan isn't
* sufficiently ordered.
*/
if (pathkeys != NIL)
{
int numsortkeys;
AttrNumber *sortColIdx;
Oid *sortOperators;
Oid *collations;
bool *nullsFirst;
/*
* Compute sort column info, and adjust subplan's tlist as needed.
* We must apply prepare_sort_from_pathkeys even to subplans that
* don't need an explicit sort, to make sure they are returning
* the same sort key columns the Append expects.
*/
subplan = prepare_sort_from_pathkeys(subplan, pathkeys,
subpath->parent->relids,
nodeSortColIdx,
false,
&numsortkeys,
&sortColIdx,
&sortOperators,
&collations,
&nullsFirst);
/*
* Check that we got the same sort key information. We just
* Assert that the sortops match, since those depend only on the
* pathkeys; but it seems like a good idea to check the sort
* column numbers explicitly, to ensure the tlists match up.
*/
Assert(numsortkeys == nodenumsortkeys);
if (memcmp(sortColIdx, nodeSortColIdx,
numsortkeys * sizeof(AttrNumber)) != 0)
elog(ERROR, "Append child's targetlist doesn't match Append");
Assert(memcmp(sortOperators, nodeSortOperators,
numsortkeys * sizeof(Oid)) == 0);
Assert(memcmp(collations, nodeCollations,
numsortkeys * sizeof(Oid)) == 0);
Assert(memcmp(nullsFirst, nodeNullsFirst,
numsortkeys * sizeof(bool)) == 0);
/* Now, insert a Sort node if subplan isn't sufficiently ordered */
if (!pathkeys_contained_in(pathkeys, subpath->pathkeys))
{
Sort *sort = make_sort(subplan, numsortkeys,
sortColIdx, sortOperators,
collations, nullsFirst);
label_sort_with_costsize(root, sort, best_path->limit_tuples);
subplan = (Plan *) sort;
}
}
subplans = lappend(subplans, subplan);
}
......@@ -1133,15 +1225,9 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
prunequal);
}
/*
* And build the Append plan. Note that if there's just one child, the
* Append is pretty useless; but we wait till setrefs.c to get rid of it.
* Doing so here doesn't work because the varno of the child scan plan
* won't match the parent-rel Vars it'll be asked to emit.
*/
plan = make_append(subplans, best_path->first_partial_path,
tlist, partpruneinfo);
plan->appendplans = subplans;
plan->first_partial_plan = best_path->first_partial_path;
plan->part_prune_info = partpruneinfo;
copy_generic_path_info(&plan->plan, (Path *) best_path);
......@@ -1266,7 +1352,6 @@ create_merge_append_plan(PlannerInfo *root, MergeAppendPath *best_path)
if (best_path->path.param_info)
{
List *prmquals = best_path->path.param_info->ppi_clauses;
prmquals = extract_actual_clauses(prmquals, false);
......@@ -5300,23 +5385,6 @@ make_foreignscan(List *qptlist,
return node;
}
static Append *
make_append(List *appendplans, int first_partial_plan,
List *tlist, PartitionPruneInfo *partpruneinfo)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
plan->targetlist = tlist;
plan->qual = NIL;
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
node->first_partial_plan = first_partial_plan;
node->part_prune_info = partpruneinfo;
return node;
}
static RecursiveUnion *
make_recursive_union(List *tlist,
Plan *lefttree,
......
......@@ -1721,7 +1721,8 @@ inheritance_planner(PlannerInfo *root)
/* Make a dummy path, cf set_dummy_rel_pathlist() */
dummy_path = (Path *) create_append_path(NULL, final_rel, NIL, NIL,
NULL, 0, false, NIL, -1);
NIL, NULL, 0, false,
NIL, -1);
/* These lists must be nonempty to make a valid ModifyTable node */
subpaths = list_make1(dummy_path);
......@@ -4003,6 +4004,7 @@ create_degenerate_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel,
grouped_rel,
paths,
NIL,
NIL,
NULL,
0,
false,
......
......@@ -617,7 +617,7 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
* Append the child results together.
*/
path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
NULL, 0, false, NIL, -1);
NIL, NULL, 0, false, NIL, -1);
/*
* For UNION ALL, we just need the Append path. For UNION, need to add
......@@ -672,7 +672,8 @@ generate_union_paths(SetOperationStmt *op, PlannerInfo *root,
ppath = (Path *)
create_append_path(root, result_rel, NIL, partial_pathlist,
NULL, parallel_workers, enable_parallel_append,
NIL, NULL,
parallel_workers, enable_parallel_append,
NIL, -1);
ppath = (Path *)
create_gather_path(root, result_rel, ppath,
......@@ -783,7 +784,7 @@ generate_nonunion_paths(SetOperationStmt *op, PlannerInfo *root,
* Append the child results together.
*/
path = (Path *) create_append_path(root, result_rel, pathlist, NIL,
NULL, 0, false, NIL, -1);
NIL, NULL, 0, false, NIL, -1);
/* Identify the grouping semantics */
groupList = generate_setop_grouplist(op, tlist);
......
......@@ -1203,12 +1203,13 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* pathnode.
*
* Note that we must handle subpaths = NIL, representing a dummy access path.
* Also, there are callers that pass root = NULL.
*/
AppendPath *
create_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths, List *partial_subpaths,
Relids required_outer,
List *pathkeys, Relids required_outer,
int parallel_workers, bool parallel_aware,
List *partitioned_rels, double rows)
{
......@@ -1242,6 +1243,7 @@ create_append_path(PlannerInfo *root,
pathnode->path.parallel_aware = parallel_aware;
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = pathkeys;
pathnode->partitioned_rels = list_copy(partitioned_rels);
/*
......@@ -1255,6 +1257,13 @@ create_append_path(PlannerInfo *root,
*/
if (pathnode->path.parallel_aware)
{
/*
* We mustn't fiddle with the order of subpaths when the Append has
* pathkeys. The order they're listed in is critical to keeping the
* pathkeys valid.
*/
Assert(pathkeys == NIL);
subpaths = list_qsort(subpaths, append_total_cost_compare);
partial_subpaths = list_qsort(partial_subpaths,
append_startup_cost_compare);
......@@ -1262,6 +1271,15 @@ create_append_path(PlannerInfo *root,
pathnode->first_partial_path = list_length(subpaths);
pathnode->subpaths = list_concat(subpaths, partial_subpaths);
/*
* Apply query-wide LIMIT if known and path is for sole base relation.
* (Handling this at this low level is a bit klugy.)
*/
if (root != NULL && bms_equal(rel->relids, root->all_baserels))
pathnode->limit_tuples = root->limit_tuples;
else
pathnode->limit_tuples = -1.0;
foreach(l, pathnode->subpaths)
{
Path *subpath = (Path *) lfirst(l);
......@@ -1278,8 +1296,9 @@ create_append_path(PlannerInfo *root,
/*
* If there's exactly one child path, the Append is a no-op and will be
* discarded later (in setrefs.c); therefore, we can inherit the child's
* size, cost, and pathkeys if any. Otherwise, it's unsorted, and we must
* do the normal costsize calculation.
* size and cost, as well as its pathkeys if any (overriding whatever the
* caller might've said). Otherwise, we must do the normal costsize
* calculation.
*/
if (list_length(pathnode->subpaths) == 1)
{
......@@ -1291,10 +1310,7 @@ create_append_path(PlannerInfo *root,
pathnode->path.pathkeys = child->pathkeys;
}
else
{
pathnode->path.pathkeys = NIL; /* unsorted if more than 1 subpath */
cost_append(pathnode);
}
/* If the caller provided a row estimate, override the computed value. */
if (rows >= 0)
......@@ -3759,7 +3775,7 @@ reparameterize_path(PlannerInfo *root, Path *path,
}
return (Path *)
create_append_path(root, rel, childpaths, partialpaths,
required_outer,
apath->path.pathkeys, required_outer,
apath->path.parallel_workers,
apath->path.parallel_aware,
apath->partitioned_rels,
......
......@@ -861,6 +861,70 @@ partition_bounds_copy(PartitionBoundInfo src,
return dest;
}
/*
* partitions_are_ordered
* Determine whether the partitions described by 'boundinfo' are ordered,
* that is partitions appearing earlier in the PartitionDesc sequence
* contain partition keys strictly less than those appearing later.
* Also, if NULL values are possible, they must come in the last
* partition defined in the PartitionDesc.
*
* If out of order, or there is insufficient info to know the order,
* then we return false.
*/
bool
partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts)
{
Assert(boundinfo != NULL);
switch (boundinfo->strategy)
{
case PARTITION_STRATEGY_RANGE:
/*
* RANGE-type partitioning guarantees that the partitions can be
* scanned in the order that they're defined in the PartitionDesc
* to provide sequential, non-overlapping ranges of tuples.
* However, if a DEFAULT partition exists then it doesn't work, as
* that could contain tuples from either below or above the
* defined range, or tuples belonging to gaps between partitions.
*/
if (!partition_bound_has_default(boundinfo))
return true;
break;
case PARTITION_STRATEGY_LIST:
/*
* LIST partitioning can also guarantee ordering, but only if the
* partitions don't accept interleaved values. We could likely
* check for this by looping over the PartitionBound's indexes
* array to check that the indexes are in order. For now, let's
* just keep it simple and just accept LIST partitioning when
* there's no DEFAULT partition, exactly one value per partition,
* and optionally a NULL partition that does not accept any other
* values. Such a NULL partition will come last in the
* PartitionDesc, and the other partitions will be properly
* ordered. This is a cheap test to make as it does not require
* any per-partition processing. Maybe we'd like to handle more
* complex cases in the future.
*/
if (partition_bound_has_default(boundinfo))
return false;
if (boundinfo->ndatums + partition_bound_accepts_nulls(boundinfo)
== nparts)
return true;
break;
default:
/* HASH, or some other strategy */
break;
}
return false;
}
/*
* check_new_partition_bound
*
......
......@@ -280,6 +280,11 @@ struct PlannerInfo
List *join_info_list; /* list of SpecialJoinInfos */
/*
* Note: for AppendRelInfos describing partitions of a partitioned table,
* we guarantee that partitions that come earlier in the partitioned
* table's PartitionDesc will appear earlier in append_rel_list.
*/
List *append_rel_list; /* list of AppendRelInfos */
List *rowMarks; /* list of PlanRowMarks */
......@@ -1363,9 +1368,9 @@ typedef struct AppendPath
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *subpaths; /* list of component Paths */
/* Index of first partial path in subpaths */
/* Index of first partial path in subpaths; list_length(subpaths) if none */
int first_partial_path;
double limit_tuples; /* hard limit on output tuples, or -1 */
} AppendPath;
#define IS_DUMMY_APPEND(p) \
......
......@@ -65,7 +65,7 @@ extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
extern AppendPath *create_append_path(PlannerInfo *root, RelOptInfo *rel,
List *subpaths, List *partial_subpaths,
Relids required_outer,
List *pathkeys, Relids required_outer,
int parallel_workers, bool parallel_aware,
List *partitioned_rels, double rows);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
......
......@@ -194,6 +194,8 @@ extern Path *get_cheapest_fractional_path_for_pathkeys(List *paths,
extern Path *get_cheapest_parallel_safe_total_inner(List *paths);
extern List *build_index_pathkeys(PlannerInfo *root, IndexOptInfo *index,
ScanDirection scandir);
extern List *build_partition_pathkeys(PlannerInfo *root, RelOptInfo *partrel,
ScanDirection scandir, bool *partialkeys);
extern List *build_expression_pathkey(PlannerInfo *root, Expr *expr,
Relids nullable_relids, Oid opno,
Relids rel, bool create_it);
......
......@@ -88,6 +88,7 @@ extern bool partition_bounds_equal(int partnatts, int16 *parttyplen,
PartitionBoundInfo b2);
extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
PartitionKey key);
extern bool partitions_are_ordered(PartitionBoundInfo boundinfo, int nparts);
extern void check_new_partition_bound(char *relname, Relation parent,
PartitionBoundSpec *spec);
extern void check_default_partition_contents(Relation parent,
......
This diff is collapsed.
......@@ -3078,14 +3078,14 @@ drop table boolp;
--
set enable_seqscan = off;
set enable_sort = off;
create table ma_test (a int) partition by range (a);
create table ma_test (a int, b int) partition by range (a);
create table ma_test_p1 partition of ma_test for values from (0) to (10);
create table ma_test_p2 partition of ma_test for values from (10) to (20);
create table ma_test_p3 partition of ma_test for values from (20) to (30);
insert into ma_test select x from generate_series(0,29) t(x);
create index on ma_test (a);
insert into ma_test select x,x from generate_series(0,29) t(x);
create index on ma_test (b);
analyze ma_test;
prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
prepare mt_q1 (int) as select a from ma_test where a >= $1 and a % 10 = 5 order by b;
-- Execute query 5 times to allow choose_custom_plan
-- to start considering a generic plan.
execute mt_q1(0);
......@@ -3132,17 +3132,15 @@ explain (analyze, costs off, summary off, timing off) execute mt_q1(15);
QUERY PLAN
-------------------------------------------------------------------------------
Merge Append (actual rows=2 loops=1)
Sort Key: ma_test_p2.a
Sort Key: ma_test_p2.b
Subplans Removed: 1
-> Index Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=1 loops=1)
Index Cond: (a >= $1)
Filter: ((a % 10) = 5)
Rows Removed by Filter: 4
-> Index Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
Index Cond: (a >= $1)
Filter: ((a % 10) = 5)
-> Index Scan using ma_test_p2_b_idx on ma_test_p2 (actual rows=1 loops=1)
Filter: ((a >= $1) AND ((a % 10) = 5))
Rows Removed by Filter: 9
(11 rows)
-> Index Scan using ma_test_p3_b_idx on ma_test_p3 (actual rows=1 loops=1)
Filter: ((a >= $1) AND ((a % 10) = 5))
Rows Removed by Filter: 9
(9 rows)
execute mt_q1(15);
a
......@@ -3155,13 +3153,12 @@ explain (analyze, costs off, summary off, timing off) execute mt_q1(25);
QUERY PLAN
-------------------------------------------------------------------------------
Merge Append (actual rows=1 loops=1)
Sort Key: ma_test_p3.a
Sort Key: ma_test_p3.b
Subplans Removed: 2
-> Index Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=1 loops=1)
Index Cond: (a >= $1)
Filter: ((a % 10) = 5)
Rows Removed by Filter: 4
(7 rows)
-> Index Scan using ma_test_p3_b_idx on ma_test_p3 (actual rows=1 loops=1)
Filter: ((a >= $1) AND ((a % 10) = 5))
Rows Removed by Filter: 9
(6 rows)
execute mt_q1(25);
a
......@@ -3174,12 +3171,11 @@ explain (analyze, costs off, summary off, timing off) execute mt_q1(35);
QUERY PLAN
------------------------------------------------------------------------
Merge Append (actual rows=0 loops=1)
Sort Key: ma_test_p1.a
Sort Key: ma_test_p1.b
Subplans Removed: 2
-> Index Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
Index Cond: (a >= $1)
Filter: ((a % 10) = 5)
(6 rows)
-> Index Scan using ma_test_p1_b_idx on ma_test_p1 (never executed)
Filter: ((a >= $1) AND ((a % 10) = 5))
(5 rows)
execute mt_q1(35);
a
......@@ -3188,23 +3184,23 @@ execute mt_q1(35);
deallocate mt_q1;
-- ensure initplan params properly prune partitions
explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(b) from ma_test_p2) order by b;
QUERY PLAN
------------------------------------------------------------------------------------------------------------
Merge Append (actual rows=20 loops=1)
Sort Key: ma_test_p1.a
Sort Key: ma_test_p1.b
InitPlan 2 (returns $1)
-> Result (actual rows=1 loops=1)
InitPlan 1 (returns $0)
-> Limit (actual rows=1 loops=1)
-> Index Scan using ma_test_p2_a_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
Index Cond: (a IS NOT NULL)
-> Index Scan using ma_test_p1_a_idx on ma_test_p1 (never executed)
Index Cond: (a >= $1)
-> Index Scan using ma_test_p2_a_idx on ma_test_p2 (actual rows=10 loops=1)
Index Cond: (a >= $1)
-> Index Scan using ma_test_p3_a_idx on ma_test_p3 (actual rows=10 loops=1)
Index Cond: (a >= $1)
-> Index Scan using ma_test_p2_b_idx on ma_test_p2 ma_test_p2_1 (actual rows=1 loops=1)
Index Cond: (b IS NOT NULL)
-> Index Scan using ma_test_p1_b_idx on ma_test_p1 (never executed)
Filter: (a >= $1)
-> Index Scan using ma_test_p2_b_idx on ma_test_p2 (actual rows=10 loops=1)
Filter: (a >= $1)
-> Index Scan using ma_test_p3_b_idx on ma_test_p3 (actual rows=10 loops=1)
Filter: (a >= $1)
(14 rows)
reset enable_seqscan;
......
......@@ -715,7 +715,6 @@ explain (costs off) select * from mcrparted where abs(b) = 5; -- scans all parti
explain (costs off) select * from mcrparted where a > -1; -- scans all partitions
explain (costs off) select * from mcrparted where a = 20 and abs(b) = 10 and c > 10; -- scans mcrparted4
explain (costs off) select * from mcrparted where a = 20 and c > 20; -- scans mcrparted3, mcrparte4, mcrparte5, mcrparted_def
drop table mcrparted;
-- check that partitioned table Appends cope with being referenced in
-- subplans
......@@ -726,3 +725,111 @@ insert into parted_minmax values (1,'12345');
explain (costs off) select min(a), max(a) from parted_minmax where b = '12345';
select min(a), max(a) from parted_minmax where b = '12345';
drop table parted_minmax;
-- Test code that uses Append nodes in place of MergeAppend when the
-- partition ordering matches the desired ordering.
create index mcrparted_a_abs_c_idx on mcrparted (a, abs(b), c);
-- MergeAppend must be used when a default partition exists
explain (costs off) select * from mcrparted order by a, abs(b), c;
drop table mcrparted_def;
-- Append is used for a RANGE partitioned table with no default
-- and no subpartitions
explain (costs off) select * from mcrparted order by a, abs(b), c;
-- Append is used with subpaths in reverse order with backwards index scans
explain (costs off) select * from mcrparted order by a desc, abs(b) desc, c desc;
-- check that Append plan is used containing a MergeAppend for sub-partitions
-- that are unordered.
drop table mcrparted5;
create table mcrparted5 partition of mcrparted for values from (20, 20, 20) to (maxvalue, maxvalue, maxvalue) partition by list (a);
create table mcrparted5a partition of mcrparted5 for values in(20);
create table mcrparted5_def partition of mcrparted5 default;
explain (costs off) select * from mcrparted order by a, abs(b), c;
drop table mcrparted5_def;
-- check that an Append plan is used and the sub-partitions are flattened
-- into the main Append when the sub-partition is unordered but contains
-- just a single sub-partition.
explain (costs off) select a, abs(b) from mcrparted order by a, abs(b), c;
-- check that Append is used when the sub-partitioned tables are pruned
-- during planning.
explain (costs off) select * from mcrparted where a < 20 order by a, abs(b), c;
create table mclparted (a int) partition by list(a);
create table mclparted1 partition of mclparted for values in(1);
create table mclparted2 partition of mclparted for values in(2);
create index on mclparted (a);
-- Ensure an Append is used for a list partition with an order by.
explain (costs off) select * from mclparted order by a;
-- Ensure a MergeAppend is used when a partition exists with interleaved
-- datums in the partition bound.
create table mclparted3_5 partition of mclparted for values in(3,5);
create table mclparted4 partition of mclparted for values in(4);
explain (costs off) select * from mclparted order by a;
drop table mclparted;
-- Ensure subplans which don't have a path with the correct pathkeys get
-- sorted correctly.
drop index mcrparted_a_abs_c_idx;
create index on mcrparted1 (a, abs(b), c);
create index on mcrparted2 (a, abs(b), c);
create index on mcrparted3 (a, abs(b), c);
create index on mcrparted4 (a, abs(b), c);
explain (costs off) select * from mcrparted where a < 20 order by a, abs(b), c limit 1;
set enable_bitmapscan = 0;
-- Ensure Append node can be used when the partition is ordered by some
-- pathkeys which were deemed redundant.
explain (costs off) select * from mcrparted where a = 10 order by a, abs(b), c;
reset enable_bitmapscan;
drop table mcrparted;
-- Ensure LIST partitions allow an Append to be used instead of a MergeAppend
create table bool_lp (b bool) partition by list(b);
create table bool_lp_true partition of bool_lp for values in(true);
create table bool_lp_false partition of bool_lp for values in(false);
create index on bool_lp (b);
explain (costs off) select * from bool_lp order by b;
drop table bool_lp;
-- Ensure const bool quals can be properly detected as redundant
create table bool_rp (b bool, a int) partition by range(b,a);
create table bool_rp_false_1k partition of bool_rp for values from (false,0) to (false,1000);
create table bool_rp_true_1k partition of bool_rp for values from (true,0) to (true,1000);
create table bool_rp_false_2k partition of bool_rp for values from (false,1000) to (false,2000);
create table bool_rp_true_2k partition of bool_rp for values from (true,1000) to (true,2000);
create index on bool_rp (b,a);
explain (costs off) select * from bool_rp where b = true order by b,a;
explain (costs off) select * from bool_rp where b = false order by b,a;
explain (costs off) select * from bool_rp where b = true order by a;
explain (costs off) select * from bool_rp where b = false order by a;
drop table bool_rp;
-- Ensure an Append scan is chosen when the partition order is a subset of
-- the required order.
create table range_parted (a int, b int, c int) partition by range(a, b);
create table range_parted1 partition of range_parted for values from (0,0) to (10,10);
create table range_parted2 partition of range_parted for values from (10,10) to (20,20);
create index on range_parted (a,b,c);
explain (costs off) select * from range_parted order by a,b,c;
explain (costs off) select * from range_parted order by a desc,b desc,c desc;
drop table range_parted;
......@@ -775,15 +775,15 @@ drop table boolp;
--
set enable_seqscan = off;
set enable_sort = off;
create table ma_test (a int) partition by range (a);
create table ma_test (a int, b int) partition by range (a);
create table ma_test_p1 partition of ma_test for values from (0) to (10);
create table ma_test_p2 partition of ma_test for values from (10) to (20);
create table ma_test_p3 partition of ma_test for values from (20) to (30);
insert into ma_test select x from generate_series(0,29) t(x);
create index on ma_test (a);
insert into ma_test select x,x from generate_series(0,29) t(x);
create index on ma_test (b);
analyze ma_test;
prepare mt_q1 (int) as select * from ma_test where a >= $1 and a % 10 = 5 order by a;
prepare mt_q1 (int) as select a from ma_test where a >= $1 and a % 10 = 5 order by b;
-- Execute query 5 times to allow choose_custom_plan
-- to start considering a generic plan.
......@@ -804,7 +804,7 @@ execute mt_q1(35);
deallocate mt_q1;
-- ensure initplan params properly prune partitions
explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(a) from ma_test_p2) order by a;
explain (analyze, costs off, summary off, timing off) select * from ma_test where a >= (select min(b) from ma_test_p2) order by b;
reset enable_seqscan;
reset enable_sort;
......
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