Parallel Append implementation
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.
Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.
The attached patch removes both of the above restrictions. There has
already been a mail thread [1]Old mail thread : /messages/by-id/9A28C8860F777E439AA12E8AEA7694F80115DEB8@BPXM15GP.gisp.nec.co.jp that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.
Attached is pgbench_create_partition.sql (derived from the one
included in the above thread) that distributes pgbench_accounts table
data into 3 partitions pgbench_account_[1-3]. The below queries use
this schema.
Consider a query such as :
select count(*) from pgbench_accounts;
Now suppose, these two partitions do not allow parallel scan :
alter table pgbench_accounts_1 set (parallel_workers=0);
alter table pgbench_accounts_2 set (parallel_workers=0);
On HEAD, due to some of the partitions having non-parallel scans, the
whole Append would be a sequential scan :
Aggregate
-> Append
-> Index Only Scan using pgbench_accounts_pkey on pgbench_accounts
-> Seq Scan on pgbench_accounts_1
-> Seq Scan on pgbench_accounts_2
-> Seq Scan on pgbench_accounts_3
Whereas, with the patch, the Append looks like this :
Finalize Aggregate
-> Gather
Workers Planned: 6
-> Partial Aggregate
-> Parallel Append
-> Parallel Seq Scan on pgbench_accounts
-> Seq Scan on pgbench_accounts_1
-> Seq Scan on pgbench_accounts_2
-> Parallel Seq Scan on pgbench_accounts_3
Above, Parallel Append is generated, and it executes all these
subplans in parallel, with 1 worker executing each of the sequential
scans, and multiple workers executing each of the parallel subplans.
======= Implementation details ========
------- Adding parallel-awareness -------
In a given worker, this Append plan node will be executing just like
the usual partial Append node. It will run a subplan until completion.
The subplan may or may not be a partial parallel-aware plan like
parallelScan. After the subplan is done, Append will choose the next
subplan. It is here where it will be different than the current
partial Append plan: it is parallel-aware. The Append nodes in the
workers will be aware that there are other Append nodes running in
parallel. The partial Append will have to coordinate with other Append
nodes while choosing the next subplan.
------- Distribution of workers --------
The coordination info is stored in a shared array, each element of
which describes the per-subplan info. This info contains the number of
workers currently executing the subplan, and the maximum number of
workers that should be executing it at the same time. For non-partial
sublans, max workers would always be 1. For choosing the next subplan,
the Append executor will sequentially iterate over the array to find a
subplan having the least number of workers currently being executed,
AND which is not already being executed by the maximum number of
workers assigned for the subplan. Once it gets one, it increments
current_workers, and releases the Spinlock, so that other workers can
choose their next subplan if they are waiting.
This way, workers would be fairly distributed across subplans.
The shared array needs to be initialized and made available to
workers. For this, we can do exactly what sequential scan does for
being parallel-aware : Using function ExecAppendInitializeDSM()
similar to ExecSeqScanInitializeDSM() in the backend to allocate the
array. Similarly, for workers, have ExecAppendInitializeWorker() to
retrieve the shared array.
-------- Generating Partial Append plan having non-partial subplans --------
In set_append_rel_pathlist(), while generating a partial path for
Append, also include the non-partial child subpaths, besides the
partial subpaths. This way, it can contain a mix of partial and
non-partial children paths. But for a given child, its path would be
either the cheapest partial path or the cheapest non-partial path.
For a non-partial child path, it will only be included if it is
parallel-safe. If there is no parallel-safe path, a partial Append
path would not be generated. This behaviour also automatically
prevents paths that have a Gather node beneath.
Finally when it comes to create a partial append path using these
child paths, we also need to store a bitmap set indicating which of
the child paths are non-partial paths. For this, have a new BitmapSet
field : Append->partial_subplans. At execution time, this will be used
to set the maximum workers for a non-partial subpath to 1.
-------- Costing -------
For calculating per-worker parallel Append path cost, it first
calculates a total of child subplan costs considering all of their
workers, and then divides it by the Append node's parallel_divisor,
similar to how parallel scan uses this "parallel_divisor".
For startup cost, it is assumed that Append would start returning
tuples when the child node having the lowest startup cost is done
setting up. So Append startup cost is equal to startup cost of the
child with minimum startup cost.
-------- Scope --------
There are two different code paths where Append path is generated.
1. One is where append rel is generated : Inheritance table, and UNION
ALL clause.
2. Second codepath is in prepunion.c. This gets executed for UNION
(without ALL) and INTERSECT/EXCEPT [ALL]. The patch does not support
Parallel Append in this scenario. It can be later taken up as
extension, once this patch is reviewed.
======= Performance =======
There is a clear benefit in case of ParallelAppend in scenarios where
one or more subplans don't have partial paths, because in such cases,
on HEAD it does not generate Partial Append. For example, the below
query took around 30 secs with the patch
(max_parallel_workers_per_gather should be 3 or more), whereas, it
took 74 secs on HEAD.
explain analyze select avg(aid) from (
select aid from pgbench_accounts_1 inner join bid_tab b using (bid)
UNION ALL
select aid from pgbench_accounts_2 inner join bid_tab using (bid)
UNION ALL
select aid from pgbench_accounts_3 inner join bid_tab using (bid)
) p;
--- With HEAD ---
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=6415493.67..6415493.67 rows=1 width=32) (actual
time=74135.821..74135.822 rows=1 loops=1)
-> Append (cost=1541552.36..6390743.54 rows=9900047 width=4)
(actual time=73829.985..74125.048 rows=100000 loops=1)
-> Hash Join (cost=1541552.36..2097249.67 rows=3300039
width=4) (actual time=25758.592..25758.592 rows=0 loops=1)
Hash Cond: (pgbench_accounts_1.bid = b.bid)
-> Seq Scan on pgbench_accounts_1
(cost=0.00..87099.39 rows=3300039 width=8) (actual time=0.118..778.097
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16 rows=50000016
width=4) (actual time=24426.433..24426.433 rows=49999902 loops=1)
Buckets: 131072 Batches: 1024 Memory Usage: 2744kB
-> Seq Scan on bid_tab b (cost=0.00..721239.16
rows=50000016 width=4) (actual time=0.105..10112.995 rows=49999902
loops=1)
-> Hash Join (cost=1541552.36..2097249.67 rows=3300039
width=4) (actual time=24063.761..24063.761 rows=0 loops=1)
Hash Cond: (pgbench_accounts_2.bid = bid_tab.bid)
-> Seq Scan on pgbench_accounts_2
(cost=0.00..87099.39 rows=3300039 width=8) (actual time=0.065..779.498
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16 rows=50000016
width=4) (actual time=22708.377..22708.377 rows=49999902 loops=1)
Buckets: 131072 Batches: 1024 Memory Usage: 2744kB
-> Seq Scan on bid_tab (cost=0.00..721239.16
rows=50000016 width=4) (actual time=0.024..9513.032 rows=49999902
loops=1)
-> Hash Join (cost=1541552.36..2097243.73 rows=3299969
width=4) (actual time=24007.628..24297.067 rows=100000 loops=1)
Hash Cond: (pgbench_accounts_3.bid = bid_tab_1.bid)
-> Seq Scan on pgbench_accounts_3
(cost=0.00..87098.69 rows=3299969 width=8) (actual time=0.049..782.230
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16 rows=50000016
width=4) (actual time=22943.413..22943.413 rows=49999902 loops=1)
Buckets: 131072 Batches: 1024 Memory Usage: 2744kB
-> Seq Scan on bid_tab bid_tab_1
(cost=0.00..721239.16 rows=50000016 width=4) (actual
time=0.022..9601.753 rows=49999902 loops=1)
Planning time: 0.366 ms
Execution time: 74138.043 ms
(22 rows)
--- With Patch ---
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------------------
Finalize Aggregate (cost=2139493.66..2139493.67 rows=1 width=32)
(actual time=29658.825..29658.825 rows=1 loops=1)
-> Gather (cost=2139493.34..2139493.65 rows=3 width=32) (actual
time=29568.957..29658.804 rows=4 loops=1)
Workers Planned: 3
Workers Launched: 3
-> Partial Aggregate (cost=2138493.34..2138493.35 rows=1
width=32) (actual time=22086.324..22086.325 rows=1 loops=4)
-> Parallel Append (cost=0.00..2130243.42
rows=3299969 width=4) (actual time=22008.945..22083.536 rows=25000
loops=4)
-> Hash Join (cost=1541552.36..2097243.73
rows=3299969 width=4) (actual time=29568.605..29568.605 rows=0
loops=1)
Hash Cond: (pgbench_accounts_1.bid = b.bid)
-> Seq Scan on pgbench_accounts_1
(cost=0.00..87098.69 rows=3299969 width=8) (actual time=0.024..841.598
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16
rows=50000016 width=4) (actual time=28134.596..28134.596 rows=49999902
loops=1)
Buckets: 131072 Batches: 1024
Memory Usage: 2744kB
-> Seq Scan on bid_tab b
(cost=0.00..721239.16 rows=50000016 width=4) (actual
time=0.076..11598.097 rows=49999902 loops=1)
-> Hash Join (cost=1541552.36..2097243.73
rows=3299969 width=4) (actual time=29127.085..29127.085 rows=0
loops=1)
Hash Cond: (pgbench_accounts_2.bid = bid_tab.bid)
-> Seq Scan on pgbench_accounts_2
(cost=0.00..87098.69 rows=3299969 width=8) (actual time=0.022..837.027
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16
rows=50000016 width=4) (actual time=27658.276..27658.276 rows=49999902
loops=1)
-> Seq Scan on bid_tab
(cost=0.00..721239.16 rows=50000016 width=4) (actual
time=0.022..11561.530 rows=49999902 loops=1)
-> Hash Join (cost=1541552.36..2097243.73
rows=3299969 width=4) (actual time=29340.081..29632.180 rows=100000
loops=1)
Hash Cond: (pgbench_accounts_3.bid = bid_tab_1.bid)
-> Seq Scan on pgbench_accounts_3
(cost=0.00..87098.69 rows=3299969 width=8) (actual time=0.027..821.875
rows=3300000 loops=1)
-> Hash (cost=721239.16..721239.16
rows=50000016 width=4) (actual time=28186.009..28186.009 rows=49999902
loops=1)
-> Seq Scan on bid_tab bid_tab_1
(cost=0.00..721239.16 rows=50000016 width=4) (actual
time=0.019..11594.461 rows=49999902 loops=1)
Planning time: 0.493 ms
Execution time: 29662.791 ms
(24 rows)
Thanks to Robert Haas and Rushabh Lathia for their valuable inputs
while working on this feature.
[1]: Old mail thread : /messages/by-id/9A28C8860F777E439AA12E8AEA7694F80115DEB8@BPXM15GP.gisp.nec.co.jp
/messages/by-id/9A28C8860F777E439AA12E8AEA7694F80115DEB8@BPXM15GP.gisp.nec.co.jp
Thanks
-Amit Khandekar
Attachments:
ParallelAppend.patchapplication/octet-stream; name=ParallelAppend.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 8a6f844..ad9ad92 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeSeqscan.h"
@@ -199,6 +200,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -247,6 +252,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecSeqScanInitializeDSM((SeqScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
@@ -724,6 +733,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
case T_SeqScanState:
ExecSeqScanInitializeWorker((SeqScanState *) planstate, toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a26bd63..a6d2d63 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct
+{
+ int pa_num_workers; /* workers currently executing the subplan */
+ int pa_max_workers; /* max workers that should run the subplan */
+} parallel_append_info;
+
+struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER];
+};
+
+typedef struct ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * For Parallel Append, AppendState->as_whichplan can have PA_INVALID_PLAN
+ * value, which indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static void exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +115,22 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ return parallel_append_next(appendstate);
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +232,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +251,14 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished",
+ MyProcPid);
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +280,17 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We have got NULL. There might be other workers still processing the
+ * last chunk of rows for this same node, but there's no point for new
+ * workers to run this node, so mark this node as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +334,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +349,233 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ int i;
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ for (i = 0; i < node->as_nplans; i++)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+
+ /* Is this a partial subplan ? */
+ if (bms_is_member(i, ((Append*)node->ps.plan)->partial_subplans))
+ {
+ /*
+ * We are distributing workers equally among subplans. So, just set
+ * the max_workers to maximum possible value.
+ */
+ padesc->pa_info[i].pa_max_workers =
+ max_parallel_workers_per_gather;
+ }
+ else
+ {
+ /*
+ * Non-partial plan essentially needs to be run by one and only
+ * one worker.
+ */
+ padesc->pa_info[i].pa_max_workers = 1;
+ }
+ }
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing this
+ * node will continue to do so, but workers looking for next nodes to pick
+ * up would skip this node after this function is called. It is possible that
+ * multiple workers call this function for the same node at the same time,
+ * because these workers were executing the same node and they finished with
+ * it at the same time. The spinlock is not for this purpose. The spinlock is
+ * used so that it does not change the pa_num_workers field while workers are
+ * choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ elog(DEBUG2, "Parallelappend : pid %d : finishing plan %d",
+ MyProcPid, whichplan);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ parallel_append_info *painfo = &padesc->pa_info[whichplan];
+
+ /* Ignore plans that are already done processing */
+ if (painfo->pa_num_workers == -1)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d"
+ " since pa_num_workers is -1",
+ MyProcPid, whichplan);
+ continue;
+ }
+
+ /* Ignore plans that are already being processed by max_workers */
+ if (painfo->pa_num_workers == painfo->pa_max_workers)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d,"
+ " since reached max_worker count %d",
+ MyProcPid, whichplan, painfo->pa_max_workers);
+ continue;
+ }
+
+ /* Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
+ }
+
+ /* Increment worker count for the chosen node, if at all we found one. */
+ if (min_whichplan != PA_INVALID_PLAN)
+ {
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+ }
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+ elog(DEBUG2, "ParallelAppend : pid %d : Chosen plan : %d",
+ MyProcPid, min_whichplan);
+
+ /*
+ * If we didn't find any node to work on, it means each subplan is either
+ * finished or has reached it's pa_max_workers. In such case, should this
+ * worker wait for some subplan to have its worker count drop below its
+ * pa_max_workers so that it can choose that subplan ? It turns out that
+ * it's not worth again finding a subplan to work on. Non-partial subplan
+ * anyway can have only one worker, and that worker will execute it to
+ * completion. For a partial subplan, if at all it reaches pa_max_workers,
+ * it's worker count will reduce only when it's workers find that there is
+ * nothing more to be executed, so there is no point taking up such node if
+ * it's worker count reduces. In conclusion, just stop executing once we
+ * don't find nodes to work on. Indicate the same by returning false.
+ */
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d973225..ee9c640 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -217,6 +217,7 @@ _copyAppend(const Append *from)
/*
* copy remainder of node
*/
+ COPY_BITMAPSET_FIELD(partial_subplans);
COPY_NODE_FIELD(appendplans);
return newnode;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 7258c03..73f47cc 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -358,6 +358,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
+ WRITE_BITMAPSET_FIELD(partial_subplans);
WRITE_NODE_FIELD(appendplans);
}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d608530..7f1c2e1 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1520,6 +1520,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
+ READ_BITMAPSET_FIELD(partial_subplans);
READ_NODE_FIELD(appendplans);
READ_DONE();
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 9753a26..62eefdb 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -98,7 +98,8 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
-static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_append_subpath(List *subpaths, Path *path,
+ Bitmapset **partial_subpaths_set);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1126,6 +1127,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
bool subpaths_valid = true;
List *partial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ Bitmapset *partial_subpath_set = NULL;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1185,14 +1187,52 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path,
+ NULL);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
- linitial(childrel->partial_pathlist));
+ linitial(childrel->partial_pathlist),
+ &partial_subpath_set);
+ }
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
+ Path *parallel_safe_path = NULL;
+ foreach(lcp, childrel->pathlist)
+ {
+ Path *child_path = (Path *) lfirst(lcp);
+ if (child_path->parallel_safe &&
+ child_path->param_info == NULL)
+ {
+ parallel_safe_path = child_path;
+ break;
+ }
+ }
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path, NULL);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1267,7 +1307,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, subpaths,
+ NULL, NULL, 0));
/*
* Consider an append of partial unordered, unparameterized partial paths.
@@ -1278,23 +1319,32 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
ListCell *lc;
int parallel_workers = 0;
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
+ /* Decide on the number of workers to request for this append path. */
foreach(lc, partial_subpaths)
{
Path *path = lfirst(lc);
- parallel_workers = Max(parallel_workers, path->parallel_workers);
+ /*
+ * partial_subpaths can have non-partial subpaths so
+ * path->parallel_workers can be 0. For such paths, allocate one
+ * worker.
+ */
+ parallel_workers +=
+ (path->parallel_workers > 0 ? path->parallel_workers : 1);
+ ereport(DEBUG2,
+ (errmsg_internal("added %d more workers for Parallel Append",
+ (path->parallel_workers > 0 ? path->parallel_workers : 1))));
}
Assert(parallel_workers > 0);
+ /* In no case use more than max_parallel_workers_per_gather. */
+ parallel_workers = Min(parallel_workers,
+ max_parallel_workers_per_gather);
+
/* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers);
+ appendpath = create_append_path(rel, partial_subpaths,
+ partial_subpath_set,
+ NULL, parallel_workers);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1341,12 +1391,13 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
subpaths_valid = false;
break;
}
- subpaths = accumulate_append_subpath(subpaths, subpath);
+ subpaths = accumulate_append_subpath(subpaths, subpath, NULL);
}
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0));
+ create_append_path(rel, subpaths,
+ NULL, required_outer, 0));
}
}
@@ -1428,9 +1479,11 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
startup_neq_total = true;
startup_subpaths =
- accumulate_append_subpath(startup_subpaths, cheapest_startup);
+ accumulate_append_subpath(startup_subpaths,
+ cheapest_startup, NULL);
total_subpaths =
- accumulate_append_subpath(total_subpaths, cheapest_total);
+ accumulate_append_subpath(total_subpaths,
+ cheapest_total, NULL);
}
/* ... and build the MergeAppend paths */
@@ -1521,6 +1574,43 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
return cheapest;
}
+/* concat_append_subpaths
+ * helper function for accumulate_append_subpath()
+ *
+ * child_partial_subpaths_set is the bitmap set to indicate which of the
+ * childpaths are partial paths. This is currently non-NULL only in case
+ * the childpaths belong to an Append path.
+ */
+static List *
+concat_append_subpaths(List *append_subpaths, List *childpaths,
+ Bitmapset **partial_subpaths_set,
+ Bitmapset *child_partial_subpaths_set)
+{
+ int i;
+ int append_subpath_len = list_length(append_subpaths);
+
+ if (partial_subpaths_set)
+ {
+ for (i = 0; i < list_length(childpaths); i++)
+ {
+ /*
+ * The child paths themselves may or may not be partial paths. The
+ * bitmapset numbers of these paths will need to be set considering
+ * that these are to be appended onto the partial_subpaths_set.
+ */
+ if (!child_partial_subpaths_set ||
+ bms_is_member(i, child_partial_subpaths_set))
+ {
+ *partial_subpaths_set = bms_add_member(*partial_subpaths_set,
+ append_subpath_len + i);
+ }
+ }
+ }
+
+ /* list_copy is important here to avoid sharing list substructure */
+ return list_concat(append_subpaths, list_copy(childpaths));
+}
+
/*
* accumulate_append_subpath
* Add a subpath to the list being built for an Append or MergeAppend
@@ -1534,26 +1624,34 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
* omitting a sort step, which seems fine: if the parent is to be an Append,
* its result would be unsorted anyway, while if the parent is to be a
* MergeAppend, there's no point in a separate sort on a child.
+ *
+ * If partial_subpaths_set is not NULL, it means we are building a
+ * partial subpaths list, and so we need to add the path (or its child paths
+ * in case it's Append or MergeAppend) into the partial_subpaths bitmap set.
*/
static List *
-accumulate_append_subpath(List *subpaths, Path *path)
+accumulate_append_subpath(List *append_subpaths, Path *path,
+ Bitmapset **partial_subpaths_set)
{
if (IsA(path, AppendPath))
{
- AppendPath *apath = (AppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(apath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((AppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ ((AppendPath*)path)->partial_subpaths);
}
else if (IsA(path, MergeAppendPath))
{
- MergeAppendPath *mpath = (MergeAppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(mpath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((MergeAppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ NULL);
}
else
- return lappend(subpaths, path);
+ return concat_append_subpaths(append_subpaths,
+ list_make1(path),
+ partial_subpaths_set,
+ NULL);
}
/*
@@ -1576,7 +1674,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 415edad..65dd027 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool enable_nestloop = true;
bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -238,23 +239,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
/* Adjust costing for parallelism, if used. */
if (path->parallel_workers > 0)
{
- double parallel_divisor = path->parallel_workers;
- double leader_contribution;
-
- /*
- * Early experience with parallel query suggests that when there is
- * only one worker, the leader often makes a very substantial
- * contribution to executing the parallel portion of the plan, but as
- * more workers are added, it does less and less, because it's busy
- * reading tuples from the workers and doing whatever non-parallel
- * post-processing is needed. By the time we reach 4 workers, the
- * leader no longer makes a meaningful contribution. Thus, for now,
- * estimate that the leader spends 30% of its time servicing each
- * worker, and the remainder executing the parallel plan.
- */
- leader_contribution = 1.0 - (0.3 * path->parallel_workers);
- if (leader_contribution > 0)
- parallel_divisor += leader_contribution;
+ double parallel_divisor;
/*
* In the case of a parallel plan, the row count needs to represent
@@ -263,6 +248,7 @@ cost_seqscan(Path *path, PlannerInfo *root,
* because they'll anticipate receiving more rows than any given copy
* will actually get.
*/
+ parallel_divisor = get_parallel_divisor(path->parallel_workers);
path->rows = clamp_row_est(path->rows / parallel_divisor);
/* The CPU cost is divided among all the workers. */
@@ -391,6 +377,36 @@ cost_gather(GatherPath *path, PlannerInfo *root,
}
/*
+ * get_parallel_divisor
+ * For given number of parallel workers, return parallel divisor, which
+ * can then be used by the caller to estimate per worker cost or per worker
+ * rows.
+ */
+int
+get_parallel_divisor(int parallel_workers)
+{
+ double parallel_divisor = parallel_workers;
+ double leader_contribution;
+
+ /*
+ * Early experience with parallel query suggests that when there is
+ * only one worker, the leader often makes a very substantial
+ * contribution to executing the parallel portion of the plan, but as
+ * more workers are added, it does less and less, because it's busy
+ * reading tuples from the workers and doing whatever non-parallel
+ * post-processing is needed. By the time we reach 4 workers, the
+ * leader no longer makes a meaningful contribution. Thus, for now,
+ * estimate that the leader spends 30% of its time servicing each
+ * worker, and the remainder executing the parallel plan.
+ */
+ leader_contribution = 1.0 - (0.3 * parallel_workers);
+ if (leader_contribution > 0)
+ parallel_divisor += leader_contribution;
+
+ return parallel_divisor;
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
@@ -1570,6 +1586,82 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, Relids required_outer)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath->parallel_workers);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path->parallel_workers);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 01d4fea..20dbc9d 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1197,7 +1197,7 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index ad49674..94b474f 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -193,7 +193,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, Bitmapset *partial_subpaths,
+ List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1001,7 +1002,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, best_path->partial_subpaths, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -4941,7 +4942,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, Bitmapset *partial_subpaths, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -4951,6 +4952,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->partial_subplans = bms_copy(partial_subpaths);
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 41dde50..1bc3ca2 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3404,10 +3404,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b714783..7169126 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -567,7 +567,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -679,7 +679,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 6d3ccfd..7ecce5a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1202,50 +1202,28 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
- ListCell *l;
pathnode->path.pathtype = T_Append;
pathnode->path.parent = rel;
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
+
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
+ pathnode->partial_subpaths = partial_subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- foreach(l, subpaths)
- {
- Path *subpath = (Path *) lfirst(l);
-
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
-
- /* All child paths must have same parameterization */
- Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
- }
+ cost_append(&pathnode->path, subpaths, required_outer);
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index a025117..a2bf746 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -894,6 +894,16 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
+
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 51c381e..1311b9c 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index d43ec56..94bbab0 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1167,6 +1168,7 @@ typedef struct ModifyTableState
/* Per partition tuple conversion map */
} ModifyTableState;
+
/* ----------------
* AppendState information
*
@@ -1174,12 +1176,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index e2fbc7d..428ca66 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -208,6 +208,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ Bitmapset *partial_subplans;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 3a1255a..7172861 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1107,6 +1107,7 @@ typedef struct AppendPath
{
Path path;
List *subpaths; /* list of component Paths */
+ Bitmapset *partial_subpaths;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 2a4df2f..ecda17f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -98,6 +99,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths, Relids required_outer);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
@@ -183,6 +185,7 @@ extern void set_cte_size_estimates(PlannerInfo *root, RelOptInfo *rel,
double cte_rows);
extern void set_foreign_size_estimates(PlannerInfo *root, RelOptInfo *rel);
extern PathTarget *set_pathtarget_cost_width(PlannerInfo *root, PathTarget *target);
+extern int get_parallel_divisor(int parallel_workers);
/*
* prototypes for clausesel.c
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 71d9154..69ddf4c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -61,8 +62,9 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index f06cfa4..858d81b 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,18 +1,19 @@
SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(11 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(12 rows)
CREATE TABLE foo2(fooid int, f2 int);
INSERT INTO foo2 VALUES(1, 11);
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 18e21b7..f6c4b41 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
On Fri, Dec 23, 2016 at 10:51 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.The attached patch removes both of the above restrictions. There has
already been a mail thread [1] that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.
The first goal requires some kind of synchronization which will allow workers
to be distributed across the subplans. The second goal requires some kind of
synchronization to prevent multiple workers from executing non-parallel
subplans. The patch uses different mechanisms to achieve the goals. If
we create two different patches addressing each goal, those may be
easier to handle.
We may want to think about a third goal: preventing too many workers
from executing the same plan. As per comment in get_parallel_divisor()
we do not see any benefit if more than 4 workers execute the same
node. So, an append node can distribute more than 4 worker nodes
equally among the available subplans. It might be better to do that as
a separate patch.
Attached is pgbench_create_partition.sql (derived from the one
included in the above thread) that distributes pgbench_accounts table
data into 3 partitions pgbench_account_[1-3]. The below queries use
this schema.Consider a query such as :
select count(*) from pgbench_accounts;Now suppose, these two partitions do not allow parallel scan :
alter table pgbench_accounts_1 set (parallel_workers=0);
alter table pgbench_accounts_2 set (parallel_workers=0);On HEAD, due to some of the partitions having non-parallel scans, the
whole Append would be a sequential scan :Aggregate
-> Append
-> Index Only Scan using pgbench_accounts_pkey on pgbench_accounts
-> Seq Scan on pgbench_accounts_1
-> Seq Scan on pgbench_accounts_2
-> Seq Scan on pgbench_accounts_3Whereas, with the patch, the Append looks like this :
Finalize Aggregate
-> Gather
Workers Planned: 6
-> Partial Aggregate
-> Parallel Append
-> Parallel Seq Scan on pgbench_accounts
-> Seq Scan on pgbench_accounts_1
-> Seq Scan on pgbench_accounts_2
-> Parallel Seq Scan on pgbench_accounts_3Above, Parallel Append is generated, and it executes all these
subplans in parallel, with 1 worker executing each of the sequential
scans, and multiple workers executing each of the parallel subplans.======= Implementation details ========
------- Adding parallel-awareness -------
In a given worker, this Append plan node will be executing just like
the usual partial Append node. It will run a subplan until completion.
The subplan may or may not be a partial parallel-aware plan like
parallelScan. After the subplan is done, Append will choose the next
subplan. It is here where it will be different than the current
partial Append plan: it is parallel-aware. The Append nodes in the
workers will be aware that there are other Append nodes running in
parallel. The partial Append will have to coordinate with other Append
nodes while choosing the next subplan.------- Distribution of workers --------
The coordination info is stored in a shared array, each element of
which describes the per-subplan info. This info contains the number of
workers currently executing the subplan, and the maximum number of
workers that should be executing it at the same time. For non-partial
sublans, max workers would always be 1. For choosing the next subplan,
the Append executor will sequentially iterate over the array to find a
subplan having the least number of workers currently being executed,
AND which is not already being executed by the maximum number of
workers assigned for the subplan. Once it gets one, it increments
current_workers, and releases the Spinlock, so that other workers can
choose their next subplan if they are waiting.This way, workers would be fairly distributed across subplans.
The shared array needs to be initialized and made available to
workers. For this, we can do exactly what sequential scan does for
being parallel-aware : Using function ExecAppendInitializeDSM()
similar to ExecSeqScanInitializeDSM() in the backend to allocate the
array. Similarly, for workers, have ExecAppendInitializeWorker() to
retrieve the shared array.-------- Generating Partial Append plan having non-partial subplans --------
In set_append_rel_pathlist(), while generating a partial path for
Append, also include the non-partial child subpaths, besides the
partial subpaths. This way, it can contain a mix of partial and
non-partial children paths. But for a given child, its path would be
either the cheapest partial path or the cheapest non-partial path.For a non-partial child path, it will only be included if it is
parallel-safe. If there is no parallel-safe path, a partial Append
path would not be generated. This behaviour also automatically
prevents paths that have a Gather node beneath.Finally when it comes to create a partial append path using these
child paths, we also need to store a bitmap set indicating which of
the child paths are non-partial paths. For this, have a new BitmapSet
field : Append->partial_subplans. At execution time, this will be used
to set the maximum workers for a non-partial subpath to 1.
We may be able to eliminate this field. Please check comment 6 below.
-------- Costing -------
For calculating per-worker parallel Append path cost, it first
calculates a total of child subplan costs considering all of their
workers, and then divides it by the Append node's parallel_divisor,
similar to how parallel scan uses this "parallel_divisor".For startup cost, it is assumed that Append would start returning
tuples when the child node having the lowest startup cost is done
setting up. So Append startup cost is equal to startup cost of the
child with minimum startup cost.-------- Scope --------
There are two different code paths where Append path is generated.
1. One is where append rel is generated : Inheritance table, and UNION
ALL clause.
2. Second codepath is in prepunion.c. This gets executed for UNION
(without ALL) and INTERSECT/EXCEPT [ALL]. The patch does not support
Parallel Append in this scenario. It can be later taken up as
extension, once this patch is reviewed.
Here are some review comments
1. struct ParallelAppendDescData is being used at other places. The declaration
style doesn't seem to be very common in the code or in the directory where the
file is located.
+struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose
next subplan */
+ parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER];
+};
Defining it like
typdef struct ParallelAppendDescData
{
slock_t pa_mutex; /* mutual exclusion to choose next
subplan */
parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER];
};
will make its use handy. Instead of struct ParallelAppendDescData, you will
need to use just ParallelAppendDescData. May be we want to rename
parallel_append_info as ParallelAppendInfo and change the style to match other
declarations.
2. The comment below refers to the constant which it describes, which looks
odd. May be it should be worded as "A special value of
AppendState::as_whichplan, to indicate no plans left to be executed.". Also
using INVALID for "no plans left ..." seems to be a misnomer.
/*
* For Parallel Append, AppendState::as_whichplan can have PA_INVALID_PLAN
* value, which indicates there are no plans left to be executed.
*/
#define PA_INVALID_PLAN -1
3. The sentence "We have got NULL", looks odd. Probably we don't need it as
it's clear from the code above that this code deals with the case when the
current subplan didn't return any row.
/*
* We have got NULL. There might be other workers still processing the
* last chunk of rows for this same node, but there's no point for new
* workers to run this node, so mark this node as finished.
*/
4. In the same comment, I guess, the word "node" refers to "subnode" and not
the node pointed by variable "node". May be you want to use word "subplan"
here.
4. set_finished()'s prologue has different indentation compared to other
functions in the file.
5. Multilevel comment starts with an empty line.
+ /* Keep track of the node with the least workers so far. For the very
6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.
7. If we consider 6, we don't need concat_append_subpaths(), but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)
{
for (i = 0; i < list_length(childpaths); i++)
{
/*
* The child paths themselves may or may not be partial paths. The
* bitmapset numbers of these paths will need to be set considering
* that these are to be appended onto the partial_subpaths_set.
*/
if (!child_partial_subpaths_set ||
bms_is_member(i, child_partial_subpaths_set))
{
*partial_subpaths_set = bms_add_member(*partial_subpaths_set,
append_subpath_len + i);
}
}
}
8.
- parallel_workers = Max(parallel_workers, path->parallel_workers);
+ /*
+ * partial_subpaths can have non-partial subpaths so
+ * path->parallel_workers can be 0. For such paths, allocate one
+ * worker.
+ */
+ parallel_workers +=
+ (path->parallel_workers > 0 ? path->parallel_workers : 1);
This looks odd. Earlier code was choosing maximum of all parallel workers,
whereas new code adds them all. E.g. if parallel_workers for subpaths is 3, 4,
3, without your change, it will pick up 4. But with your change it will pick
10. I think, you intend to write this as
parallel_workers = Max(parallel_workers, path->parallel_workers ?
path->parallel_workers : 1);
If you do that probably you don't need since parallel_workers are never set
more than max_parallel_workers_per_gather.
+ /* In no case use more than max_parallel_workers_per_gather. */
+ parallel_workers = Min(parallel_workers,
+ max_parallel_workers_per_gather);
+
9. Shouldn't this funciton return double?
int
get_parallel_divisor(int parallel_workers)
9. In get_parallel_divisor(), if parallel_worker is 0 i.e. it's a partial path
the return value will be 2, which isn't true. This function is being called for
all the subpaths to get the original number of rows and costs of partial paths.
I think we don't need to call this function on subpaths which are not partial
paths or make it work parallel_workers = 0.
10. We should probably move the parallel_safe calculation out of cost_append().
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
11. This check shouldn't be part of cost_append().
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.
13. No braces required for single line block
+ /* Increment worker count for the chosen node, if at all we found one. */
+ if (min_whichplan != PA_INVALID_PLAN)
+ {
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+ }
14. exec_append_scan_first() is a one-liner function, should we just inline it?
15. This patch replaces exec_append_initialize_next() with
exec_append_scan_first(). The earlier function was handling backward and
forward scans separately, but the later function doesn't do that. Why?
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Thanks Ashutosh for the feedback.
On 6 January 2017 at 17:04, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Fri, Dec 23, 2016 at 10:51 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.The attached patch removes both of the above restrictions. There has
already been a mail thread [1] that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.The first goal requires some kind of synchronization which will allow workers
to be distributed across the subplans. The second goal requires some kind of
synchronization to prevent multiple workers from executing non-parallel
subplans. The patch uses different mechanisms to achieve the goals. If
we create two different patches addressing each goal, those may be
easier to handle.
Goal A : Allow non-partial subpaths in Partial Append.
Goal B : Distribute workers across the Append subplans.
Both of these require some kind of synchronization while choosing the
next subplan. So, goal B is achieved by doing all the synchronization
stuff. And implementation of goal A requires that goal B is
implemented. So there is a dependency between these two goals. While
implementing goal B, we should keep in mind that it should also work
for goal A; it does not make sense later changing the synchronization
logic in goal A.
I am ok with splitting the patch into 2 patches :
a) changes required for goal A
b) changes required for goal B.
But I think we should split it only when we are ready to commit them
(commit for B, immediately followed by commit for A). Until then, we
should consider both of these together because they are interconnected
as explained above.
We may want to think about a third goal: preventing too many workers
from executing the same plan. As per comment in get_parallel_divisor()
we do not see any benefit if more than 4 workers execute the same
node. So, an append node can distribute more than 4 worker nodes
equally among the available subplans. It might be better to do that as
a separate patch.
I think that comment is for calculating leader contribution. It does
not say that 4 workers is too many workers in general.
But yes, I agree, and I have it in mind as the next improvement.
Basically, it does not make sense to give more than 3 workers to a
subplan when parallel_workers for that subplan are 3. For e.g., if
gather max workers is 10, and we have 2 Append subplans s1 and s2 with
parallel workers 3 and 5 respectively. Then, with the current patch,
it will distribute 4 workers to each of these workers. What we should
do is : once both of the subplans get 3 workers each, we should give
the 7th and 8th worker to s2.
Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.
Here are some review comments
I will handle the other comments, but first, just a quick response to
some important ones :
6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.
Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).
7. If we consider 6, we don't need concat_append_subpaths(), but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)
{
for (i = 0; i < list_length(childpaths); i++)
{
/*
* The child paths themselves may or may not be partial paths. The
* bitmapset numbers of these paths will need to be set considering
* that these are to be appended onto the partial_subpaths_set.
*/
if (!child_partial_subpaths_set ||
bms_is_member(i, child_partial_subpaths_set))
{
*partial_subpaths_set = bms_add_member(*partial_subpaths_set,
append_subpath_len + i);
}
}
}
Again, for the reason mentioned above, we will defer this point for now.
8. - parallel_workers = Max(parallel_workers, path->parallel_workers); + /* + * partial_subpaths can have non-partial subpaths so + * path->parallel_workers can be 0. For such paths, allocate one + * worker. + */ + parallel_workers += + (path->parallel_workers > 0 ? path->parallel_workers : 1);This looks odd. Earlier code was choosing maximum of all parallel workers,
whereas new code adds them all. E.g. if parallel_workers for subpaths is 3, 4,
3, without your change, it will pick up 4. But with your change it will pick
10. I think, you intend to write this as
parallel_workers = Max(parallel_workers, path->parallel_workers ?
path->parallel_workers : 1);
The intention is to add all workers, because a parallel-aware Append
is going to need them in order to make the subplans run with their
full capacity in parallel. So with subpaths with 3, 4, and 3 workers,
the Append path will need 10 workers. If it allocates 4 workers, its
not sufficient : Each of them would get only 1 worker, or max 2. In
the existing code, 4 is correct, because all the workers are going to
execute the same subplan node at a time.
9. Shouldn't this funciton return double?
int
get_parallel_divisor(int parallel_workers)
Yes, right, I will do that.
9. In get_parallel_divisor(), if parallel_worker is 0 i.e. it's a partial path
the return value will be 2, which isn't true. This function is being called for
all the subpaths to get the original number of rows and costs of partial paths.
I think we don't need to call this function on subpaths which are not partial
paths or make it work parallel_workers = 0.
I didn't understand this. I checked again get_parallel_divisor()
function code. I think it will return 1 if parallel_workers is 0. But
I may be missing something.
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.
WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.
13. No braces required for single line block + /* Increment worker count for the chosen node, if at all we found one. */ + if (min_whichplan != PA_INVALID_PLAN) + { + padesc->pa_info[min_whichplan].pa_num_workers++; + }14. exec_append_scan_first() is a one-liner function, should we just inline it?
15. This patch replaces exec_append_initialize_next() with
exec_append_scan_first(). The earlier function was handling backward and
forward scans separately, but the later function doesn't do that. Why?
I will come to these and some other ones later.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Jan 16, 2017 at 9:49 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Thanks Ashutosh for the feedback.
On 6 January 2017 at 17:04, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Fri, Dec 23, 2016 at 10:51 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.The attached patch removes both of the above restrictions. There has
already been a mail thread [1] that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.The first goal requires some kind of synchronization which will allow workers
to be distributed across the subplans. The second goal requires some kind of
synchronization to prevent multiple workers from executing non-parallel
subplans. The patch uses different mechanisms to achieve the goals. If
we create two different patches addressing each goal, those may be
easier to handle.Goal A : Allow non-partial subpaths in Partial Append.
Goal B : Distribute workers across the Append subplans.
Both of these require some kind of synchronization while choosing the
next subplan. So, goal B is achieved by doing all the synchronization
stuff. And implementation of goal A requires that goal B is
implemented. So there is a dependency between these two goals. While
implementing goal B, we should keep in mind that it should also work
for goal A; it does not make sense later changing the synchronization
logic in goal A.I am ok with splitting the patch into 2 patches :
a) changes required for goal A
b) changes required for goal B.
But I think we should split it only when we are ready to commit them
(commit for B, immediately followed by commit for A). Until then, we
should consider both of these together because they are interconnected
as explained above.
For B, we need to know, how much gain that brings and in which cases.
If that gain is not worth the complexity added, we may have to defer
Goal B. Goal A would certainly be useful since it will improve
performance of the targetted cases. The synchronization required for
Goal A is simpler than that of B and thus if we choose to implement
only A, we can live with a simpler synchronization.
BTW, Right now, the patch does not consider non-partial paths for a
child which has partial paths. Do we know, for sure, that a path
containing partial paths for a child, which has it, is always going to
be cheaper than the one which includes non-partial path. If not,
should we build another paths which contains non-partial paths for all
child relations. This sounds like a 0/1 knapsack problem.
Here are some review comments
I will handle the other comments, but first, just a quick response to
some important ones :6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).
It will be better if we can fetch that information from each subpath
when creating the plan. As I have explained before, a path is minimal
structure, which should be easily disposable, when throwing away the
path.
7. If we consider 6, we don't need concat_append_subpaths(), but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)
{
for (i = 0; i < list_length(childpaths); i++)
{
/*
* The child paths themselves may or may not be partial paths. The
* bitmapset numbers of these paths will need to be set considering
* that these are to be appended onto the partial_subpaths_set.
*/
if (!child_partial_subpaths_set ||
bms_is_member(i, child_partial_subpaths_set))
{
*partial_subpaths_set = bms_add_member(*partial_subpaths_set,
append_subpath_len + i);
}
}
}Again, for the reason mentioned above, we will defer this point for now.
Ok.
8. - parallel_workers = Max(parallel_workers, path->parallel_workers); + /* + * partial_subpaths can have non-partial subpaths so + * path->parallel_workers can be 0. For such paths, allocate one + * worker. + */ + parallel_workers += + (path->parallel_workers > 0 ? path->parallel_workers : 1);This looks odd. Earlier code was choosing maximum of all parallel workers,
whereas new code adds them all. E.g. if parallel_workers for subpaths is 3, 4,
3, without your change, it will pick up 4. But with your change it will pick
10. I think, you intend to write this as
parallel_workers = Max(parallel_workers, path->parallel_workers ?
path->parallel_workers : 1);The intention is to add all workers, because a parallel-aware Append
is going to need them in order to make the subplans run with their
full capacity in parallel. So with subpaths with 3, 4, and 3 workers,
the Append path will need 10 workers. If it allocates 4 workers, its
not sufficient : Each of them would get only 1 worker, or max 2. In
the existing code, 4 is correct, because all the workers are going to
execute the same subplan node at a time.
Ok, makes sense if we take up Goal B.
9. In get_parallel_divisor(), if parallel_worker is 0 i.e. it's a partial path
the return value will be 2, which isn't true. This function is being called for
all the subpaths to get the original number of rows and costs of partial paths.
I think we don't need to call this function on subpaths which are not partial
paths or make it work parallel_workers = 0.I didn't understand this. I checked again get_parallel_divisor()
function code. I think it will return 1 if parallel_workers is 0. But
I may be missing something.
Sorry, I also don't understand why I had that comment. For some
reason, I thought we are sending 1 when parallel_workers = 0 to
get_parallel_divisor(). But I don't understand why I thought so.
Anyway, I will provide better explanation next time I bounce against
this.
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.
The current parallel mechanism is in sync with that costing. Each
worker is supposed to take the same burden, hence the same (average)
cost. But it will change when a single worker has to scan an entire
child relation and different child relations have different sizes.
Thanks for working on the comments.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi Amit,
On 2016/12/23 14:21, Amit Khandekar wrote:
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.The attached patch removes both of the above restrictions. There has
already been a mail thread [1] that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.
I was looking at the executor portion of this patch and I noticed that in
exec_append_initialize_next():
if (appendstate->as_padesc)
return parallel_append_next(appendstate);
/*
* Not parallel-aware. Fine, just go on to the next subplan in the
* appropriate direction.
*/
if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
appendstate->as_whichplan++;
else
appendstate->as_whichplan--;
which seems to mean that executing Append in parallel mode disregards the
scan direction. I am not immediately sure what implications that has, so
I checked what heap scan does when executing in parallel mode, and found
this in heapgettup():
else if (backward)
{
/* backward parallel scan not supported */
Assert(scan->rs_parallel == NULL);
Perhaps, AppendState.as_padesc would not have been set if scan direction
is backward, because parallel mode would be disabled for the whole query
in that case (PlannerGlobal.parallelModeOK = false). Maybe add an
Assert() similar to one in heapgettup().
Thanks,
Amit
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jan 17, 2017 at 2:40 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
Hi Amit,
On 2016/12/23 14:21, Amit Khandekar wrote:
Currently an Append plan node does not execute its subplans in
parallel. There is no distribution of workers across its subplans. The
second subplan starts running only after the first subplan finishes,
although the individual subplans may be running parallel scans.Secondly, we create a partial Append path for an appendrel, but we do
that only if all of its member subpaths are partial paths. If one or
more of the subplans is a non-parallel path, there will be only a
non-parallel Append. So whatever node is sitting on top of Append is
not going to do a parallel plan; for example, a select count(*) won't
divide it into partial aggregates if the underlying Append is not
partial.The attached patch removes both of the above restrictions. There has
already been a mail thread [1] that discusses an approach suggested by
Robert Haas for implementing this feature. This patch uses this same
approach.I was looking at the executor portion of this patch and I noticed that in
exec_append_initialize_next():if (appendstate->as_padesc)
return parallel_append_next(appendstate);/*
* Not parallel-aware. Fine, just go on to the next subplan in the
* appropriate direction.
*/
if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
appendstate->as_whichplan++;
else
appendstate->as_whichplan--;which seems to mean that executing Append in parallel mode disregards the
scan direction. I am not immediately sure what implications that has, so
I checked what heap scan does when executing in parallel mode, and found
this in heapgettup():else if (backward)
{
/* backward parallel scan not supported */
Assert(scan->rs_parallel == NULL);Perhaps, AppendState.as_padesc would not have been set if scan direction
is backward, because parallel mode would be disabled for the whole query
in that case (PlannerGlobal.parallelModeOK = false). Maybe add an
Assert() similar to one in heapgettup().
There have been some reviews, but the patch has not been updated in
two weeks. Marking as "returned with feedback".
--
Michael
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
We may want to think about a third goal: preventing too many workers
from executing the same plan. As per comment in get_parallel_divisor()
we do not see any benefit if more than 4 workers execute the same
node. So, an append node can distribute more than 4 worker nodes
equally among the available subplans. It might be better to do that as
a separate patch.I think that comment is for calculating leader contribution. It does
not say that 4 workers is too many workers in general.But yes, I agree, and I have it in mind as the next improvement.
Basically, it does not make sense to give more than 3 workers to a
subplan when parallel_workers for that subplan are 3. For e.g., if
gather max workers is 10, and we have 2 Append subplans s1 and s2 with
parallel workers 3 and 5 respectively. Then, with the current patch,
it will distribute 4 workers to each of these workers. What we should
do is : once both of the subplans get 3 workers each, we should give
the 7th and 8th worker to s2.Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.
Attached is v2 patch, which implements above. Now Append plan node
stores a list of per-subplan max worker count, rather than the
Bitmapset. But still Bitmapset turned out to be necessary for
AppendPath. More details are in the subsequent comments.
Goal A : Allow non-partial subpaths in Partial Append.
Goal B : Distribute workers across the Append subplans.
Both of these require some kind of synchronization while choosing the
next subplan. So, goal B is achieved by doing all the synchronization
stuff. And implementation of goal A requires that goal B is
implemented. So there is a dependency between these two goals. While
implementing goal B, we should keep in mind that it should also work
for goal A; it does not make sense later changing the synchronization
logic in goal A.I am ok with splitting the patch into 2 patches :
a) changes required for goal A
b) changes required for goal B.
But I think we should split it only when we are ready to commit them
(commit for B, immediately followed by commit for A). Until then, we
should consider both of these together because they are interconnected
as explained above.For B, we need to know, how much gain that brings and in which cases.
If that gain is not worth the complexity added, we may have to defer
Goal B. Goal A would certainly be useful since it will improve
performance of the targetted cases. The synchronization required for
Goal A is simpler than that of B and thus if we choose to implement
only A, we can live with a simpler synchronization.
For Goal A , the logic for a worker synchronously choosing a subplan will be :
Go the next subplan. If that subplan has not already assigned max
workers, choose this subplan, otherwise, go the next subplan, and so
on.
For Goal B , the logic will be :
Among the subplans which are yet to achieve max workers, choose the
subplan with the minimum number of workers currently assigned.
I don't think there is a significant difference between the complexity
of the above two algorithms. So I think here the complexity does not
look like a factor based on which we can choose the particular logic.
We should choose the logic which has more potential for benefits. The
logic for goal B will work for goal A as well. And secondly, if the
subplans are using their own different system resources, the resource
contention might be less. One case is : all subplans using different
disks. Second case is : some of the subplans may be using a foreign
scan, so it would start using foreign server resources sooner. These
benefits apply when the Gather max workers count is not sufficient for
running all the subplans in their full capacity. If they are
sufficient, then the workers will be distributed over the subplans
using both the logics. Just the order of assignments of workers to
subplans will be different.
Also, I don't see a disadvantage if we follow the logic of Goal B.
BTW, Right now, the patch does not consider non-partial paths for a
child which has partial paths. Do we know, for sure, that a path
containing partial paths for a child, which has it, is always going to
be cheaper than the one which includes non-partial path. If not,
should we build another paths which contains non-partial paths for all
child relations. This sounds like a 0/1 knapsack problem.
I didn't quite get this. We do create a non-partial Append path using
non-partial child paths anyways.
Here are some review comments
I will handle the other comments, but first, just a quick response to
some important ones :6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).It will be better if we can fetch that information from each subpath
when creating the plan. As I have explained before, a path is minimal
structure, which should be easily disposable, when throwing away the
path.
Now in the v2 patch, we store per-subplan worker count. But still, we
cannot use the path->parallel_workers to determine whether it's a
partial path. This is because even for a non-partial path, it seems
the parallel_workers can be non-zero. For e.g., in
create_subqueryscan_path(), it sets path->parallel_workers to
subpath->parallel_workers. But this path is added as a non-partial
path. So we need a separate info as to which of the subpaths in Append
path are partial subpaths. So in the v2 patch, I continued to use
Bitmapset in AppendPath. But in Append plan node, number of workers is
calculated using this bitmapset. Check the new function
get_append_num_workers().
7. If we consider 6, we don't need concat_append_subpaths(),
As explained above, I have kept the BitmapSet for AppendPath.
but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)
I will get back on this after more thought.
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.The current parallel mechanism is in sync with that costing. Each
worker is supposed to take the same burden, hence the same (average)
cost. But it will change when a single worker has to scan an entire
child relation and different child relations have different sizes.
I gave more thought on this. Considering each subplan has different
number of workers, I think it makes sense to calculate average
per-worker cost even in parallel Append. In case of non-partial
subplan, a single worker will execute it, but it will next choose
another subplan. So on average each worker is going to process the
same number of rows, and also the same amount of CPU. And that amount
of CPU cost and rows cost should be calculated by taking the total
count and dividing it by number of workers (parallel_divsor actually).
Here are some review comments
1. struct ParallelAppendDescData is being used at other places. The declaration style doesn't seem to be very common in the code or in the directory where the file is located. +struct ParallelAppendDescData +{ + slock_t pa_mutex; /* mutual exclusion to choose next subplan */ + parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER]; +}; Defining it like typdef struct ParallelAppendDescData { slock_t pa_mutex; /* mutual exclusion to choose next subplan */ parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER]; }; will make its use handy. Instead of struct ParallelAppendDescData, you will need to use just ParallelAppendDescData. May be we want to rename parallel_append_info as ParallelAppendInfo and change the style to match other declarations.2. The comment below refers to the constant which it describes, which looks
odd. May be it should be worded as "A special value of
AppendState::as_whichplan, to indicate no plans left to be executed.". Also
using INVALID for "no plans left ..." seems to be a misnomer.
/*
* For Parallel Append, AppendState::as_whichplan can have PA_INVALID_PLAN
* value, which indicates there are no plans left to be executed.
*/
#define PA_INVALID_PLAN -13. The sentence "We have got NULL", looks odd. Probably we don't need it as
it's clear from the code above that this code deals with the case when the
current subplan didn't return any row.
/*
* We have got NULL. There might be other workers still processing the
* last chunk of rows for this same node, but there's no point for new
* workers to run this node, so mark this node as finished.
*/
4. In the same comment, I guess, the word "node" refers to "subnode" and not
the node pointed by variable "node". May be you want to use word "subplan"
here.4. set_finished()'s prologue has different indentation compared to other
functions in the file.5. Multilevel comment starts with an empty line.
+ /* Keep track of the node with the least workers so far. For the very
Done 1. to 5. above, as per your suggestions.
9. Shouldn't this funciton return double?
int
get_parallel_divisor(int parallel_workers)
v2 patch is rebased on latest master branch, which already contains
this function returning double.
10. We should probably move the parallel_safe calculation out of cost_append(). + path->parallel_safe = path->parallel_safe && + subpath->parallel_safe;11. This check shouldn't be part of cost_append(). + /* All child paths must have same parameterization */ + Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
Yet to handle the above ones.
Attachments:
ParallelAppend_v2.patchapplication/octet-stream; name=ParallelAppend_v2.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..0b50ab9 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeSeqscan.h"
@@ -201,6 +202,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -249,6 +254,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecSeqScanInitializeDSM((SeqScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
@@ -725,6 +734,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
case T_SeqScanState:
ExecSeqScanInitializeWorker((SeqScanState *) planstate, toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..97bfc89 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,48 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendInfo
+{
+ int pa_num_workers; /* workers currently executing the subplan */
+ int pa_max_workers; /* max workers that should run the subplan */
+} ParallelAppendInfo;
+
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +116,22 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ return parallel_append_next(appendstate);
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +233,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +252,14 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished",
+ MyProcPid);
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +281,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +336,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +351,223 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+ List *num_workers_list = ((Append*)node->ps.plan)->num_workers;
+ ListCell *lc;
+ int i;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ Assert(node->as_nplans == list_length(num_workers_list));
+
+ i = 0;
+ foreach(lc, num_workers_list)
+ {
+ /* Initialize the max workers count for each subplan. */
+ padesc->pa_info[i].pa_max_workers = lfirst_int(lc);
+
+ /*
+ * Also, initialize current number of workers. Just setting all the
+ * number of workers to 0 is enough. The logic of choosing the next
+ * plan in workers will take care of initializing everything else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+
+ i++;
+ }
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ elog(DEBUG2, "Parallelappend : pid %d : finishing plan %d",
+ MyProcPid, whichplan);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
+
+ /* Ignore plans that are already done processing */
+ if (painfo->pa_num_workers == -1)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d"
+ " since pa_num_workers is -1",
+ MyProcPid, whichplan);
+ continue;
+ }
+
+ /* Ignore plans that are already being processed by max_workers */
+ if (painfo->pa_num_workers == painfo->pa_max_workers)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d,"
+ " since reached max_worker count %d",
+ MyProcPid, whichplan, painfo->pa_max_workers);
+ continue;
+ }
+
+ /*
+ * Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
+ }
+
+ /* Increment worker count for the chosen node, if at all we found one. */
+ if (min_whichplan != PA_INVALID_PLAN)
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+ elog(DEBUG2, "ParallelAppend : pid %d : Chosen plan : %d",
+ MyProcPid, min_whichplan);
+
+ /*
+ * If we didn't find any node to work on, it means each subplan is either
+ * finished or has reached it's pa_max_workers. In such case, should this
+ * worker wait for some subplan to have its worker count drop below its
+ * pa_max_workers so that it can choose that subplan ? It turns out that
+ * it's not worth again finding a subplan to work on. Non-partial subplan
+ * anyway can have only one worker, and that worker will execute it to
+ * completion. For a partial subplan, if at all it reaches pa_max_workers,
+ * it's worker count will reduce only when it's workers find that there is
+ * nothing more to be executed, so there is no point taking up such node if
+ * it's worker count reduces. In conclusion, just stop executing once we
+ * don't find nodes to work on. Indicate the same by returning false.
+ */
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 30d733e..cf8d7d1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_NODE_FIELD(num_workers);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1560ac3..38e13e0 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_NODE_FIELD(num_workers);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index dcfa6ee..8d0cda4 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1539,6 +1539,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_NODE_FIELD(num_workers);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..c85271f 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -98,7 +98,8 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
-static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_append_subpath(List *subpaths, Path *path,
+ Bitmapset **partial_subpaths_set);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1173,6 +1174,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
bool subpaths_valid = true;
List *partial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ Bitmapset *partial_subpath_set = NULL;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1232,14 +1234,52 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path,
+ NULL);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
- linitial(childrel->partial_pathlist));
+ linitial(childrel->partial_pathlist),
+ &partial_subpath_set);
+ }
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
+ Path *parallel_safe_path = NULL;
+ foreach(lcp, childrel->pathlist)
+ {
+ Path *child_path = (Path *) lfirst(lcp);
+ if (child_path->parallel_safe &&
+ child_path->param_info == NULL)
+ {
+ parallel_safe_path = child_path;
+ break;
+ }
+ }
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path, NULL);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1314,7 +1354,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, subpaths,
+ NULL, NULL, 0));
/*
* Consider an append of partial unordered, unparameterized partial paths.
@@ -1322,26 +1363,15 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
-
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
+ int parallel_workers;
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths,
+ partial_subpath_set,
+ NULL);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers);
+ appendpath = create_append_path(rel, partial_subpaths,
+ partial_subpath_set,
+ NULL, parallel_workers);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1388,12 +1418,13 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
subpaths_valid = false;
break;
}
- subpaths = accumulate_append_subpath(subpaths, subpath);
+ subpaths = accumulate_append_subpath(subpaths, subpath, NULL);
}
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0));
+ create_append_path(rel, subpaths,
+ NULL, required_outer, 0));
}
}
@@ -1475,9 +1506,11 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
startup_neq_total = true;
startup_subpaths =
- accumulate_append_subpath(startup_subpaths, cheapest_startup);
+ accumulate_append_subpath(startup_subpaths,
+ cheapest_startup, NULL);
total_subpaths =
- accumulate_append_subpath(total_subpaths, cheapest_total);
+ accumulate_append_subpath(total_subpaths,
+ cheapest_total, NULL);
}
/* ... and build the MergeAppend paths */
@@ -1568,6 +1601,43 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
return cheapest;
}
+/* concat_append_subpaths
+ * helper function for accumulate_append_subpath()
+ *
+ * child_partial_subpaths_set is the bitmap set to indicate which of the
+ * childpaths are partial paths. This is currently non-NULL only in case
+ * the childpaths belong to an Append path.
+ */
+static List *
+concat_append_subpaths(List *append_subpaths, List *childpaths,
+ Bitmapset **partial_subpaths_set,
+ Bitmapset *child_partial_subpaths_set)
+{
+ int i;
+ int append_subpath_len = list_length(append_subpaths);
+
+ if (partial_subpaths_set)
+ {
+ for (i = 0; i < list_length(childpaths); i++)
+ {
+ /*
+ * The child paths themselves may or may not be partial paths. The
+ * bitmapset numbers of these paths will need to be set considering
+ * that these are to be appended onto the partial_subpaths_set.
+ */
+ if (!child_partial_subpaths_set ||
+ bms_is_member(i, child_partial_subpaths_set))
+ {
+ *partial_subpaths_set = bms_add_member(*partial_subpaths_set,
+ append_subpath_len + i);
+ }
+ }
+ }
+
+ /* list_copy is important here to avoid sharing list substructure */
+ return list_concat(append_subpaths, list_copy(childpaths));
+}
+
/*
* accumulate_append_subpath
* Add a subpath to the list being built for an Append or MergeAppend
@@ -1581,26 +1651,34 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
* omitting a sort step, which seems fine: if the parent is to be an Append,
* its result would be unsorted anyway, while if the parent is to be a
* MergeAppend, there's no point in a separate sort on a child.
+ *
+ * If partial_subpaths_set is not NULL, it means we are building a
+ * partial subpaths list, and so we need to add the path (or its child paths
+ * in case it's Append or MergeAppend) into the partial_subpaths bitmap set.
*/
static List *
-accumulate_append_subpath(List *subpaths, Path *path)
+accumulate_append_subpath(List *append_subpaths, Path *path,
+ Bitmapset **partial_subpaths_set)
{
if (IsA(path, AppendPath))
{
- AppendPath *apath = (AppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(apath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((AppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ ((AppendPath*)path)->partial_subpaths);
}
else if (IsA(path, MergeAppendPath))
{
- MergeAppendPath *mpath = (MergeAppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(mpath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((MergeAppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ NULL);
}
else
- return lappend(subpaths, path);
+ return concat_append_subpaths(append_subpaths,
+ list_make1(path),
+ partial_subpaths_set,
+ NULL);
}
/*
@@ -1623,7 +1701,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 458f139..974d12d 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool enable_nestloop = true;
bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1552,6 +1553,82 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, Relids required_outer)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6f3c20b..37e755b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1197,7 +1197,7 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index fae1f67..b25d53c 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/placeholder.h"
#include "optimizer/plancat.h"
@@ -194,7 +195,7 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, List *num_workers, List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -962,6 +963,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
Append *plan;
List *tlist = build_path_tlist(root, &best_path->path);
List *subplans = NIL;
+ List *num_workers_list;
ListCell *subpaths;
/*
@@ -1000,6 +1002,11 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
subplans = lappend(subplans, subplan);
}
+ /* Get a list of number of workers for each of the subplans */
+ (void) get_append_num_workers(best_path->subpaths,
+ best_path->partial_subpaths,
+ &num_workers_list);
+
/*
* XXX ideally, if there's just one child, we'd not bother to generate an
* Append node but just return the single child. At the moment this does
@@ -1007,7 +1014,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, num_workers_list, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5009,7 +5016,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, List *num_workers, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5019,6 +5026,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->num_workers = num_workers;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4b5902f..d397d1f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3394,10 +3394,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 06e843d..847c4b9 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..e2ead44 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1195,6 +1195,49 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ * Optionally return the list of per-subplan worker count through
+ * 'per_subplan_workers'
+ */
+int
+get_append_num_workers(List *subpaths, Bitmapset *partial_subpaths_set,
+ List **per_subplan_workers)
+{
+ ListCell *lc;
+ int total_workers = 0;
+ int subplan_workers;
+ int i = 0;
+
+ if (per_subplan_workers)
+ *per_subplan_workers = NIL;
+
+ foreach(lc, subpaths)
+ {
+ Path *subpath = lfirst(lc);
+
+ if (bms_is_member(i, partial_subpaths_set))
+ subplan_workers = subpath->parallel_workers;
+ else
+ subplan_workers = 1;
+
+ if (per_subplan_workers)
+ {
+ *per_subplan_workers =
+ lappend_int(*per_subplan_workers, subplan_workers);
+ }
+ total_workers += subplan_workers;
+ i++;
+ }
+
+ /* In no case use more than max_parallel_workers_per_gather. */
+ total_workers = Min(total_workers,
+ max_parallel_workers_per_gather);
+
+ return total_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1202,50 +1245,28 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
- ListCell *l;
pathnode->path.pathtype = T_Append;
pathnode->path.parent = rel;
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
+
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
+ pathnode->partial_subpaths = partial_subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- foreach(l, subpaths)
- {
- Path *subpath = (Path *) lfirst(l);
-
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
-
- /* All child paths must have same parameterization */
- Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
- }
+ cost_append(&pathnode->path, subpaths, required_outer);
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 74ca4e7..3588f09 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -895,6 +895,16 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
+
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..a21b16d 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1180,12 +1181,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..8c06ee0 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ List *num_workers;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..ac0ff70 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1116,6 +1116,7 @@ typedef struct AppendPath
{
Path path;
List *subpaths; /* list of component Paths */
+ Bitmapset *partial_subpaths;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 39376ec..ecbda74 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -98,6 +99,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths, Relids required_outer);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..425d7b9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -61,8 +62,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *subpaths,
+ Bitmapset *partial_subpaths_set, List **per_subplan_workers);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/rangefuncs.out b/src/test/regress/expected/rangefuncs.out
index 56481de..92439fe 100644
--- a/src/test/regress/expected/rangefuncs.out
+++ b/src/test/regress/expected/rangefuncs.out
@@ -1,18 +1,19 @@
SELECT name, setting FROM pg_settings WHERE name LIKE 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(11 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(12 rows)
CREATE TABLE foo2(fooid int, f2 int);
INSERT INTO foo2 VALUES(1, 11);
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 18e21b7..f6c4b41 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
v2 patch was not rebased over the latest master branch commits. Please
refer to the attached ParallelAppend_v3.patch, instead.
On 6 February 2017 at 11:06, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
We may want to think about a third goal: preventing too many workers
from executing the same plan. As per comment in get_parallel_divisor()
we do not see any benefit if more than 4 workers execute the same
node. So, an append node can distribute more than 4 worker nodes
equally among the available subplans. It might be better to do that as
a separate patch.I think that comment is for calculating leader contribution. It does
not say that 4 workers is too many workers in general.But yes, I agree, and I have it in mind as the next improvement.
Basically, it does not make sense to give more than 3 workers to a
subplan when parallel_workers for that subplan are 3. For e.g., if
gather max workers is 10, and we have 2 Append subplans s1 and s2 with
parallel workers 3 and 5 respectively. Then, with the current patch,
it will distribute 4 workers to each of these workers. What we should
do is : once both of the subplans get 3 workers each, we should give
the 7th and 8th worker to s2.Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.Attached is v2 patch, which implements above. Now Append plan node
stores a list of per-subplan max worker count, rather than the
Bitmapset. But still Bitmapset turned out to be necessary for
AppendPath. More details are in the subsequent comments.Goal A : Allow non-partial subpaths in Partial Append.
Goal B : Distribute workers across the Append subplans.
Both of these require some kind of synchronization while choosing the
next subplan. So, goal B is achieved by doing all the synchronization
stuff. And implementation of goal A requires that goal B is
implemented. So there is a dependency between these two goals. While
implementing goal B, we should keep in mind that it should also work
for goal A; it does not make sense later changing the synchronization
logic in goal A.I am ok with splitting the patch into 2 patches :
a) changes required for goal A
b) changes required for goal B.
But I think we should split it only when we are ready to commit them
(commit for B, immediately followed by commit for A). Until then, we
should consider both of these together because they are interconnected
as explained above.For B, we need to know, how much gain that brings and in which cases.
If that gain is not worth the complexity added, we may have to defer
Goal B. Goal A would certainly be useful since it will improve
performance of the targetted cases. The synchronization required for
Goal A is simpler than that of B and thus if we choose to implement
only A, we can live with a simpler synchronization.For Goal A , the logic for a worker synchronously choosing a subplan will be :
Go the next subplan. If that subplan has not already assigned max
workers, choose this subplan, otherwise, go the next subplan, and so
on.
For Goal B , the logic will be :
Among the subplans which are yet to achieve max workers, choose the
subplan with the minimum number of workers currently assigned.I don't think there is a significant difference between the complexity
of the above two algorithms. So I think here the complexity does not
look like a factor based on which we can choose the particular logic.
We should choose the logic which has more potential for benefits. The
logic for goal B will work for goal A as well. And secondly, if the
subplans are using their own different system resources, the resource
contention might be less. One case is : all subplans using different
disks. Second case is : some of the subplans may be using a foreign
scan, so it would start using foreign server resources sooner. These
benefits apply when the Gather max workers count is not sufficient for
running all the subplans in their full capacity. If they are
sufficient, then the workers will be distributed over the subplans
using both the logics. Just the order of assignments of workers to
subplans will be different.Also, I don't see a disadvantage if we follow the logic of Goal B.
BTW, Right now, the patch does not consider non-partial paths for a
child which has partial paths. Do we know, for sure, that a path
containing partial paths for a child, which has it, is always going to
be cheaper than the one which includes non-partial path. If not,
should we build another paths which contains non-partial paths for all
child relations. This sounds like a 0/1 knapsack problem.I didn't quite get this. We do create a non-partial Append path using
non-partial child paths anyways.Here are some review comments
I will handle the other comments, but first, just a quick response to
some important ones :6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).It will be better if we can fetch that information from each subpath
when creating the plan. As I have explained before, a path is minimal
structure, which should be easily disposable, when throwing away the
path.Now in the v2 patch, we store per-subplan worker count. But still, we
cannot use the path->parallel_workers to determine whether it's a
partial path. This is because even for a non-partial path, it seems
the parallel_workers can be non-zero. For e.g., in
create_subqueryscan_path(), it sets path->parallel_workers to
subpath->parallel_workers. But this path is added as a non-partial
path. So we need a separate info as to which of the subpaths in Append
path are partial subpaths. So in the v2 patch, I continued to use
Bitmapset in AppendPath. But in Append plan node, number of workers is
calculated using this bitmapset. Check the new function
get_append_num_workers().7. If we consider 6, we don't need concat_append_subpaths(),
As explained above, I have kept the BitmapSet for AppendPath.
but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)I will get back on this after more thought.
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.The current parallel mechanism is in sync with that costing. Each
worker is supposed to take the same burden, hence the same (average)
cost. But it will change when a single worker has to scan an entire
child relation and different child relations have different sizes.I gave more thought on this. Considering each subplan has different
number of workers, I think it makes sense to calculate average
per-worker cost even in parallel Append. In case of non-partial
subplan, a single worker will execute it, but it will next choose
another subplan. So on average each worker is going to process the
same number of rows, and also the same amount of CPU. And that amount
of CPU cost and rows cost should be calculated by taking the total
count and dividing it by number of workers (parallel_divsor actually).Here are some review comments
1. struct ParallelAppendDescData is being used at other places. The declaration style doesn't seem to be very common in the code or in the directory where the file is located. +struct ParallelAppendDescData +{ + slock_t pa_mutex; /* mutual exclusion to choose next subplan */ + parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER]; +}; Defining it like typdef struct ParallelAppendDescData { slock_t pa_mutex; /* mutual exclusion to choose next subplan */ parallel_append_info pa_info[FLEXIBLE_ARRAY_MEMBER]; }; will make its use handy. Instead of struct ParallelAppendDescData, you will need to use just ParallelAppendDescData. May be we want to rename parallel_append_info as ParallelAppendInfo and change the style to match other declarations.2. The comment below refers to the constant which it describes, which looks
odd. May be it should be worded as "A special value of
AppendState::as_whichplan, to indicate no plans left to be executed.". Also
using INVALID for "no plans left ..." seems to be a misnomer.
/*
* For Parallel Append, AppendState::as_whichplan can have PA_INVALID_PLAN
* value, which indicates there are no plans left to be executed.
*/
#define PA_INVALID_PLAN -13. The sentence "We have got NULL", looks odd. Probably we don't need it as
it's clear from the code above that this code deals with the case when the
current subplan didn't return any row.
/*
* We have got NULL. There might be other workers still processing the
* last chunk of rows for this same node, but there's no point for new
* workers to run this node, so mark this node as finished.
*/
4. In the same comment, I guess, the word "node" refers to "subnode" and not
the node pointed by variable "node". May be you want to use word "subplan"
here.4. set_finished()'s prologue has different indentation compared to other
functions in the file.5. Multilevel comment starts with an empty line.
+ /* Keep track of the node with the least workers so far. For the veryDone 1. to 5. above, as per your suggestions.
9. Shouldn't this funciton return double?
int
get_parallel_divisor(int parallel_workers)v2 patch is rebased on latest master branch, which already contains
this function returning double.10. We should probably move the parallel_safe calculation out of cost_append(). + path->parallel_safe = path->parallel_safe && + subpath->parallel_safe;11. This check shouldn't be part of cost_append(). + /* All child paths must have same parameterization */ + Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));Yet to handle the above ones.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
ParallelAppend_v3.patchapplication/octet-stream; name=ParallelAppend_v3.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index e01fe6d..0b50ab9 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeSeqscan.h"
@@ -201,6 +202,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -249,6 +254,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecSeqScanInitializeDSM((SeqScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
@@ -725,6 +734,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
case T_SeqScanState:
ExecSeqScanInitializeWorker((SeqScanState *) planstate, toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..97bfc89 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,48 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendInfo
+{
+ int pa_num_workers; /* workers currently executing the subplan */
+ int pa_max_workers; /* max workers that should run the subplan */
+} ParallelAppendInfo;
+
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +116,22 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ return parallel_append_next(appendstate);
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +233,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +252,14 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished",
+ MyProcPid);
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +281,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +336,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +351,223 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+ List *num_workers_list = ((Append*)node->ps.plan)->num_workers;
+ ListCell *lc;
+ int i;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ Assert(node->as_nplans == list_length(num_workers_list));
+
+ i = 0;
+ foreach(lc, num_workers_list)
+ {
+ /* Initialize the max workers count for each subplan. */
+ padesc->pa_info[i].pa_max_workers = lfirst_int(lc);
+
+ /*
+ * Also, initialize current number of workers. Just setting all the
+ * number of workers to 0 is enough. The logic of choosing the next
+ * plan in workers will take care of initializing everything else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+
+ i++;
+ }
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ elog(DEBUG2, "Parallelappend : pid %d : finishing plan %d",
+ MyProcPid, whichplan);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
+
+ /* Ignore plans that are already done processing */
+ if (painfo->pa_num_workers == -1)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d"
+ " since pa_num_workers is -1",
+ MyProcPid, whichplan);
+ continue;
+ }
+
+ /* Ignore plans that are already being processed by max_workers */
+ if (painfo->pa_num_workers == painfo->pa_max_workers)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d,"
+ " since reached max_worker count %d",
+ MyProcPid, whichplan, painfo->pa_max_workers);
+ continue;
+ }
+
+ /*
+ * Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
+ }
+
+ /* Increment worker count for the chosen node, if at all we found one. */
+ if (min_whichplan != PA_INVALID_PLAN)
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+ elog(DEBUG2, "ParallelAppend : pid %d : Chosen plan : %d",
+ MyProcPid, min_whichplan);
+
+ /*
+ * If we didn't find any node to work on, it means each subplan is either
+ * finished or has reached it's pa_max_workers. In such case, should this
+ * worker wait for some subplan to have its worker count drop below its
+ * pa_max_workers so that it can choose that subplan ? It turns out that
+ * it's not worth again finding a subplan to work on. Non-partial subplan
+ * anyway can have only one worker, and that worker will execute it to
+ * completion. For a partial subplan, if at all it reaches pa_max_workers,
+ * it's worker count will reduce only when it's workers find that there is
+ * nothing more to be executed, so there is no point taking up such node if
+ * it's worker count reduces. In conclusion, just stop executing once we
+ * don't find nodes to work on. Indicate the same by returning false.
+ */
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 30d733e..cf8d7d1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_NODE_FIELD(num_workers);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1560ac3..38e13e0 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_NODE_FIELD(num_workers);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index dcfa6ee..8d0cda4 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1539,6 +1539,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_NODE_FIELD(num_workers);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 5c18987..c85271f 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -98,7 +98,8 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
-static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_append_subpath(List *subpaths, Path *path,
+ Bitmapset **partial_subpaths_set);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1173,6 +1174,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
bool subpaths_valid = true;
List *partial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ Bitmapset *partial_subpath_set = NULL;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1232,14 +1234,52 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path,
+ NULL);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
- linitial(childrel->partial_pathlist));
+ linitial(childrel->partial_pathlist),
+ &partial_subpath_set);
+ }
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
+ Path *parallel_safe_path = NULL;
+ foreach(lcp, childrel->pathlist)
+ {
+ Path *child_path = (Path *) lfirst(lcp);
+ if (child_path->parallel_safe &&
+ child_path->param_info == NULL)
+ {
+ parallel_safe_path = child_path;
+ break;
+ }
+ }
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path, NULL);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1314,7 +1354,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, subpaths,
+ NULL, NULL, 0));
/*
* Consider an append of partial unordered, unparameterized partial paths.
@@ -1322,26 +1363,15 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
-
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
+ int parallel_workers;
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths,
+ partial_subpath_set,
+ NULL);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers);
+ appendpath = create_append_path(rel, partial_subpaths,
+ partial_subpath_set,
+ NULL, parallel_workers);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1388,12 +1418,13 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
subpaths_valid = false;
break;
}
- subpaths = accumulate_append_subpath(subpaths, subpath);
+ subpaths = accumulate_append_subpath(subpaths, subpath, NULL);
}
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0));
+ create_append_path(rel, subpaths,
+ NULL, required_outer, 0));
}
}
@@ -1475,9 +1506,11 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
startup_neq_total = true;
startup_subpaths =
- accumulate_append_subpath(startup_subpaths, cheapest_startup);
+ accumulate_append_subpath(startup_subpaths,
+ cheapest_startup, NULL);
total_subpaths =
- accumulate_append_subpath(total_subpaths, cheapest_total);
+ accumulate_append_subpath(total_subpaths,
+ cheapest_total, NULL);
}
/* ... and build the MergeAppend paths */
@@ -1568,6 +1601,43 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
return cheapest;
}
+/* concat_append_subpaths
+ * helper function for accumulate_append_subpath()
+ *
+ * child_partial_subpaths_set is the bitmap set to indicate which of the
+ * childpaths are partial paths. This is currently non-NULL only in case
+ * the childpaths belong to an Append path.
+ */
+static List *
+concat_append_subpaths(List *append_subpaths, List *childpaths,
+ Bitmapset **partial_subpaths_set,
+ Bitmapset *child_partial_subpaths_set)
+{
+ int i;
+ int append_subpath_len = list_length(append_subpaths);
+
+ if (partial_subpaths_set)
+ {
+ for (i = 0; i < list_length(childpaths); i++)
+ {
+ /*
+ * The child paths themselves may or may not be partial paths. The
+ * bitmapset numbers of these paths will need to be set considering
+ * that these are to be appended onto the partial_subpaths_set.
+ */
+ if (!child_partial_subpaths_set ||
+ bms_is_member(i, child_partial_subpaths_set))
+ {
+ *partial_subpaths_set = bms_add_member(*partial_subpaths_set,
+ append_subpath_len + i);
+ }
+ }
+ }
+
+ /* list_copy is important here to avoid sharing list substructure */
+ return list_concat(append_subpaths, list_copy(childpaths));
+}
+
/*
* accumulate_append_subpath
* Add a subpath to the list being built for an Append or MergeAppend
@@ -1581,26 +1651,34 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
* omitting a sort step, which seems fine: if the parent is to be an Append,
* its result would be unsorted anyway, while if the parent is to be a
* MergeAppend, there's no point in a separate sort on a child.
+ *
+ * If partial_subpaths_set is not NULL, it means we are building a
+ * partial subpaths list, and so we need to add the path (or its child paths
+ * in case it's Append or MergeAppend) into the partial_subpaths bitmap set.
*/
static List *
-accumulate_append_subpath(List *subpaths, Path *path)
+accumulate_append_subpath(List *append_subpaths, Path *path,
+ Bitmapset **partial_subpaths_set)
{
if (IsA(path, AppendPath))
{
- AppendPath *apath = (AppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(apath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((AppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ ((AppendPath*)path)->partial_subpaths);
}
else if (IsA(path, MergeAppendPath))
{
- MergeAppendPath *mpath = (MergeAppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(mpath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((MergeAppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ NULL);
}
else
- return lappend(subpaths, path);
+ return concat_append_subpaths(append_subpaths,
+ list_make1(path),
+ partial_subpaths_set,
+ NULL);
}
/*
@@ -1623,7 +1701,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a43daa7..895e5e6 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool enable_nestloop = true;
bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1515,6 +1516,82 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, Relids required_outer)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6f3c20b..37e755b 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1197,7 +1197,7 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..d3491d5 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/placeholder.h"
#include "optimizer/plancat.h"
@@ -194,7 +195,7 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, List *num_workers, List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -962,6 +963,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
Append *plan;
List *tlist = build_path_tlist(root, &best_path->path);
List *subplans = NIL;
+ List *num_workers_list;
ListCell *subpaths;
/*
@@ -1000,6 +1002,11 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
subplans = lappend(subplans, subplan);
}
+ /* Get a list of number of workers for each of the subplans */
+ (void) get_append_num_workers(best_path->subpaths,
+ best_path->partial_subpaths,
+ &num_workers_list);
+
/*
* XXX ideally, if there's just one child, we'd not bother to generate an
* Append node but just return the single child. At the moment this does
@@ -1007,7 +1014,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, num_workers_list, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5009,7 +5016,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, List *num_workers, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5019,6 +5026,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->num_workers = num_workers;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 881742f..329e7d8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3380,10 +3380,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 06e843d..847c4b9 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f440875..e2ead44 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1195,6 +1195,49 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ * Optionally return the list of per-subplan worker count through
+ * 'per_subplan_workers'
+ */
+int
+get_append_num_workers(List *subpaths, Bitmapset *partial_subpaths_set,
+ List **per_subplan_workers)
+{
+ ListCell *lc;
+ int total_workers = 0;
+ int subplan_workers;
+ int i = 0;
+
+ if (per_subplan_workers)
+ *per_subplan_workers = NIL;
+
+ foreach(lc, subpaths)
+ {
+ Path *subpath = lfirst(lc);
+
+ if (bms_is_member(i, partial_subpaths_set))
+ subplan_workers = subpath->parallel_workers;
+ else
+ subplan_workers = 1;
+
+ if (per_subplan_workers)
+ {
+ *per_subplan_workers =
+ lappend_int(*per_subplan_workers, subplan_workers);
+ }
+ total_workers += subplan_workers;
+ i++;
+ }
+
+ /* In no case use more than max_parallel_workers_per_gather. */
+ total_workers = Min(total_workers,
+ max_parallel_workers_per_gather);
+
+ return total_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1202,50 +1245,28 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
- ListCell *l;
pathnode->path.pathtype = T_Append;
pathnode->path.parent = rel;
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
+
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
+ pathnode->partial_subpaths = partial_subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- foreach(l, subpaths)
- {
- Path *subpath = (Path *) lfirst(l);
-
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
-
- /* All child paths must have same parameterization */
- Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
- }
+ cost_append(&pathnode->path, subpaths, required_outer);
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index c53aede..97e5a39 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -895,6 +895,16 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
+
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f9bcdd6..a21b16d 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1180,12 +1181,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..8c06ee0 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ List *num_workers;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 643be54..ac0ff70 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1116,6 +1116,7 @@ typedef struct AppendPath
{
Path path;
List *subpaths; /* list of component Paths */
+ Bitmapset *partial_subpaths;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 0e68264..875d3ed 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -98,6 +99,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths, Relids required_outer);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 7b41317..425d7b9 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -61,8 +62,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *subpaths,
+ Bitmapset *partial_subpaths_set, List **per_subplan_workers);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 18e21b7..f6c4b41 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..7a303fa 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(11 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(12 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
On Mon, Feb 6, 2017 at 11:06 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
We may want to think about a third goal: preventing too many workers
from executing the same plan. As per comment in get_parallel_divisor()
we do not see any benefit if more than 4 workers execute the same
node. So, an append node can distribute more than 4 worker nodes
equally among the available subplans. It might be better to do that as
a separate patch.I think that comment is for calculating leader contribution. It does
not say that 4 workers is too many workers in general.But yes, I agree, and I have it in mind as the next improvement.
Basically, it does not make sense to give more than 3 workers to a
subplan when parallel_workers for that subplan are 3. For e.g., if
gather max workers is 10, and we have 2 Append subplans s1 and s2 with
parallel workers 3 and 5 respectively. Then, with the current patch,
it will distribute 4 workers to each of these workers. What we should
do is : once both of the subplans get 3 workers each, we should give
the 7th and 8th worker to s2.Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.Attached is v2 patch, which implements above. Now Append plan node
stores a list of per-subplan max worker count, rather than the
Bitmapset. But still Bitmapset turned out to be necessary for
AppendPath. More details are in the subsequent comments.Goal A : Allow non-partial subpaths in Partial Append.
Goal B : Distribute workers across the Append subplans.
Both of these require some kind of synchronization while choosing the
next subplan. So, goal B is achieved by doing all the synchronization
stuff. And implementation of goal A requires that goal B is
implemented. So there is a dependency between these two goals. While
implementing goal B, we should keep in mind that it should also work
for goal A; it does not make sense later changing the synchronization
logic in goal A.I am ok with splitting the patch into 2 patches :
a) changes required for goal A
b) changes required for goal B.
But I think we should split it only when we are ready to commit them
(commit for B, immediately followed by commit for A). Until then, we
should consider both of these together because they are interconnected
as explained above.For B, we need to know, how much gain that brings and in which cases.
If that gain is not worth the complexity added, we may have to defer
Goal B. Goal A would certainly be useful since it will improve
performance of the targetted cases. The synchronization required for
Goal A is simpler than that of B and thus if we choose to implement
only A, we can live with a simpler synchronization.For Goal A , the logic for a worker synchronously choosing a subplan will be :
Go the next subplan. If that subplan has not already assigned max
workers, choose this subplan, otherwise, go the next subplan, and so
on.
Right, at a given time, we have to remember only the next plan to
assign worker to. That's simpler than remembering the number of
workers for each subplan and updating those concurrently. That's why I
am saying synchronization for A is simpler than that of B.
For Goal B , the logic will be :
Among the subplans which are yet to achieve max workers, choose the
subplan with the minimum number of workers currently assigned.I don't think there is a significant difference between the complexity
of the above two algorithms. So I think here the complexity does not
look like a factor based on which we can choose the particular logic.
We should choose the logic which has more potential for benefits. The
logic for goal B will work for goal A as well. And secondly, if the
subplans are using their own different system resources, the resource
contention might be less. One case is : all subplans using different
disks. Second case is : some of the subplans may be using a foreign
scan, so it would start using foreign server resources sooner. These
benefits apply when the Gather max workers count is not sufficient for
running all the subplans in their full capacity. If they are
sufficient, then the workers will be distributed over the subplans
using both the logics. Just the order of assignments of workers to
subplans will be different.Also, I don't see a disadvantage if we follow the logic of Goal B.
Do we have any performance measurements where we see that Goal B
performs better than Goal A, in such a situation? Do we have any
performance measurement comparing these two approaches in other
situations. If implementation for Goal B beats that of Goal A always,
we can certainly implement it directly. But it may not. Also,
separating patches for Goal A and Goal B might make reviews easier.
BTW, Right now, the patch does not consider non-partial paths for a
child which has partial paths. Do we know, for sure, that a path
containing partial paths for a child, which has it, is always going to
be cheaper than the one which includes non-partial path. If not,
should we build another paths which contains non-partial paths for all
child relations. This sounds like a 0/1 knapsack problem.I didn't quite get this. We do create a non-partial Append path using
non-partial child paths anyways.
Let's say a given child-relation has both partial and non-partial
paths, your approach would always pick up a partial path. But now that
parallel append can handle non-partial paths as well, it may happen
that picking up non-partial path instead of partial one when both are
available gives an overall better performance. Have we ruled out that
possibility.
Here are some review comments
I will handle the other comments, but first, just a quick response to
some important ones :6. By looking at parallel_worker field of a path, we can say whether it's
partial or not. We probably do not require to maintain a bitmap for that at in
the Append path. The bitmap can be constructed, if required, at the time of
creating the partial append plan. The reason to take this small step is 1. we
want to minimize our work at the time of creating paths, 2. while freeing a
path in add_path, we don't free the internal structures, in this case the
Bitmap, which will waste memory if the path is not chosen while planning.Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).It will be better if we can fetch that information from each subpath
when creating the plan. As I have explained before, a path is minimal
structure, which should be easily disposable, when throwing away the
path.Now in the v2 patch, we store per-subplan worker count. But still, we
cannot use the path->parallel_workers to determine whether it's a
partial path. This is because even for a non-partial path, it seems
the parallel_workers can be non-zero. For e.g., in
create_subqueryscan_path(), it sets path->parallel_workers to
subpath->parallel_workers. But this path is added as a non-partial
path. So we need a separate info as to which of the subpaths in Append
path are partial subpaths. So in the v2 patch, I continued to use
Bitmapset in AppendPath. But in Append plan node, number of workers is
calculated using this bitmapset. Check the new function
get_append_num_workers().
If the subpath from childrel->partial_pathlist, then we set the
corresponding bit in the bitmap. Now we can infer that for any path if
that path is found in path->parent->partial_pathlist. Since the code
always chooses the first partial path, the search in partial_pathlist
should not affect performance. So, we can avoid maintaining a bitmap
in the path and keep accumulating it when collapsing append paths.
7. If we consider 6, we don't need concat_append_subpaths(),
As explained above, I have kept the BitmapSet for AppendPath.
but still here are
some comments about that function. Instead of accepting two separate arguments
childpaths and child_partial_subpaths_set, which need to be in sync, we can
just pass the path which contains both of those. In the same following code may
be optimized by adding a utility function to Bitmapset, which advances
all members
by given offset and using that function with bms_union() to merge the
bitmapset e.g.
bms_union(*partial_subpaths_set,
bms_advance_members(bms_copy(child_partial_subpaths_set), append_subpath_len));
if (partial_subpaths_set)I will get back on this after more thought.
Another possibility, you could use a loop like offset_relid_set(),
using bms_next_member(). That way we could combine the for loop and
bms_is_member() call into a loop over bms_next_member().
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.The current parallel mechanism is in sync with that costing. Each
worker is supposed to take the same burden, hence the same (average)
cost. But it will change when a single worker has to scan an entire
child relation and different child relations have different sizes.I gave more thought on this. Considering each subplan has different
number of workers, I think it makes sense to calculate average
per-worker cost even in parallel Append. In case of non-partial
subplan, a single worker will execute it, but it will next choose
another subplan. So on average each worker is going to process the
same number of rows, and also the same amount of CPU. And that amount
of CPU cost and rows cost should be calculated by taking the total
count and dividing it by number of workers (parallel_divsor actually).
That's not entirely true. Consider N child relations with chosen paths
with costs C1, C2, ... CN which are very very different. If there are
N workers, the total cost should correspond to the highest of the
costs of subpaths, since no worker will execute more than one plan.
The unfortunate worker which executes the costliest path would take
the longest time. The cost of parallel append should reflect that. The
patch does not make any attempt to distribute workers based on the
actual load, so such skews should be considered into costing. I don't
think we can do anything to the condition I explained.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Feb 6, 2017 at 12:36 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.Attached is v2 patch, which implements above. Now Append plan node
stores a list of per-subplan max worker count, rather than the
Bitmapset. But still Bitmapset turned out to be necessary for
AppendPath. More details are in the subsequent comments.
Keep in mind that, for a non-partial path, the cap of 1 worker for
that subplan is a hard limit. Anything more will break the world.
But for a partial plan, the limit -- whether 1 or otherwise -- is a
soft limit. It may not help much to route more workers to that node,
and conceivably it could even hurt, but it shouldn't yield any
incorrect result. I'm not sure it's a good idea to conflate those two
things. For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high. Had those two tables been
a single unpartitioned table, I would have picked 4 or 5 workers, not
7. On the other hand, if I pick parallel_workers of 4 or 5 for the
Parallel Append, and I finish with the larger table first, I think I
might as well throw all 4 of those workers at the smaller table even
though it would normally have only used 3 workers. Having the extra
1-2 workers exist does not seem better.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Feb 14, 2017 at 12:05 PM, Robert Haas <robertmhaas@gmail.com> wrote:
Having the extra
1-2 workers exist does not seem better.
Err, exit, not exist.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 14 February 2017 at 22:35, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Feb 6, 2017 at 12:36 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Now that I think of that, I think for implementing above, we need to
keep track of per-subplan max_workers in the Append path; and with
that, the bitmap will be redundant. Instead, it can be replaced with
max_workers. Let me check if it is easy to do that. We don't want to
have the bitmap if we are sure it would be replaced by some other data
structure.Attached is v2 patch, which implements above. Now Append plan node
stores a list of per-subplan max worker count, rather than the
Bitmapset. But still Bitmapset turned out to be necessary for
AppendPath. More details are in the subsequent comments.Keep in mind that, for a non-partial path, the cap of 1 worker for
that subplan is a hard limit. Anything more will break the world.
But for a partial plan, the limit -- whether 1 or otherwise -- is a
soft limit. It may not help much to route more workers to that node,
and conceivably it could even hurt, but it shouldn't yield any
incorrect result. I'm not sure it's a good idea to conflate those two
things.
Yes, the logic that I used in the patch assumes that
"Path->parallel_workers field not only suggests how many workers to
allocate, but also prevents allocation of too many workers for that
path". For seqscan path, this field is calculated based on the
relation pages count. I believe the theory is that, too many workers
might even slow down the parallel scan. And the same theory would be
applied for calculating other types of low-level paths like index
scan.
The only reason I combined the soft limit and the hard limit is
because it is not necessary to have two different fields. But of
course this is again under the assumption that allocating more than
parallel_workers would never improve the speed, in fact it can even
slow it down.
Do we have such a case currently where the actual number of workers
launched turns out to be *more* than Path->parallel_workers ?
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high. Had those two tables been
a single unpartitioned table, I would have picked 4 or 5 workers, not
7. On the other hand, if I pick parallel_workers of 4 or 5 for the
Parallel Append, and I finish with the larger table first, I think I
might as well throw all 4 of those workers at the smaller table even
though it would normally have only used 3 workers.
Having the extra 1-2 workers exit does not seem better.
It is here, where I didn't understand exactly why would we want to
assign these extra workers to a subplan which tells use that it is
already being run by 'parallel_workers' number of workers.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 14 February 2017 at 22:35, Robert Haas <robertmhaas@gmail.com> wrote:
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high.
In the patch, in such case, 7 workers are indeed selected for Parallel
Append path, so that both the subplans are able to execute in parallel
with their full worker capacity. Are you suggesting that we should not
?
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Feb 15, 2017 at 2:33 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
The only reason I combined the soft limit and the hard limit is
because it is not necessary to have two different fields. But of
course this is again under the assumption that allocating more than
parallel_workers would never improve the speed, in fact it can even
slow it down.
That could be true in extreme cases, but in general I think it's probably false.
Do we have such a case currently where the actual number of workers
launched turns out to be *more* than Path->parallel_workers ?
No.
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high. Had those two tables been
a single unpartitioned table, I would have picked 4 or 5 workers, not
7. On the other hand, if I pick parallel_workers of 4 or 5 for the
Parallel Append, and I finish with the larger table first, I think I
might as well throw all 4 of those workers at the smaller table even
though it would normally have only used 3 workers.Having the extra 1-2 workers exit does not seem better.
It is here, where I didn't understand exactly why would we want to
assign these extra workers to a subplan which tells use that it is
already being run by 'parallel_workers' number of workers.
The decision to use fewer workers for a smaller scan isn't really
because we think that using more workers will cause a regression.
It's because we think it may not help very much, and because it's not
worth firing up a ton of workers for a relatively small scan given
that workers are a limited resource. I think once we've got a bunch
of workers started, we might as well try to use them.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Feb 15, 2017 at 4:43 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 14 February 2017 at 22:35, Robert Haas <robertmhaas@gmail.com> wrote:
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high.In the patch, in such case, 7 workers are indeed selected for Parallel
Append path, so that both the subplans are able to execute in parallel
with their full worker capacity. Are you suggesting that we should not
?
Absolutely. I think that's going to be way too many workers. Imagine
that there are 100 child tables and each one is big enough to qualify
for 2 or 3 workers. No matter what value the user has selected for
max_parallel_workers_per_gather, they should not get a scan involving
200 workers.
What I was thinking about is something like this:
1. First, take the maximum parallel_workers value from among all the children.
2. Second, compute log2(num_children)+1 and round up. So, for 1
child, 1; for 2 children, 2; for 3-4 children, 3; for 5-8 children, 4;
for 9-16 children, 5, and so on.
3. Use as the number of parallel workers for the children the maximum
of the value computed in step 1 and the value computed in step 2.
With this approach, a plan with 100 children qualifies for 8 parallel
workers (unless one of the children individually qualifies for some
larger number, or unless max_parallel_workers_per_gather is set to a
smaller value). That seems fairly reasonable to me.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Feb 15, 2017 at 6:40 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Feb 15, 2017 at 4:43 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 14 February 2017 at 22:35, Robert Haas <robertmhaas@gmail.com> wrote:
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high.In the patch, in such case, 7 workers are indeed selected for Parallel
Append path, so that both the subplans are able to execute in parallel
with their full worker capacity. Are you suggesting that we should not
?Absolutely. I think that's going to be way too many workers. Imagine
that there are 100 child tables and each one is big enough to qualify
for 2 or 3 workers. No matter what value the user has selected for
max_parallel_workers_per_gather, they should not get a scan involving
200 workers.
If the user is ready throw 200 workers and if the subplans can use
them to speed up the query 200 times (obviously I am exaggerating),
why not to use those? When the user set
max_parallel_workers_per_gather to that high a number, he meant it to
be used by a gather, and that's what we should be doing.
What I was thinking about is something like this:
1. First, take the maximum parallel_workers value from among all the children.
2. Second, compute log2(num_children)+1 and round up. So, for 1
child, 1; for 2 children, 2; for 3-4 children, 3; for 5-8 children, 4;
for 9-16 children, 5, and so on.
Can you please explain the rationale behind this maths?
3. Use as the number of parallel workers for the children the maximum
of the value computed in step 1 and the value computed in step 2.With this approach, a plan with 100 children qualifies for 8 parallel
workers (unless one of the children individually qualifies for some
larger number, or unless max_parallel_workers_per_gather is set to a
smaller value). That seems fairly reasonable to me.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 15 February 2017 at 18:40, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Feb 15, 2017 at 4:43 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 14 February 2017 at 22:35, Robert Haas <robertmhaas@gmail.com> wrote:
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high.In the patch, in such case, 7 workers are indeed selected for Parallel
Append path, so that both the subplans are able to execute in parallel
with their full worker capacity. Are you suggesting that we should not
?Absolutely. I think that's going to be way too many workers. Imagine
that there are 100 child tables and each one is big enough to qualify
for 2 or 3 workers. No matter what value the user has selected for
max_parallel_workers_per_gather, they should not get a scan involving
200 workers.What I was thinking about is something like this:
1. First, take the maximum parallel_workers value from among all the children.
2. Second, compute log2(num_children)+1 and round up. So, for 1
child, 1; for 2 children, 2; for 3-4 children, 3; for 5-8 children, 4;
for 9-16 children, 5, and so on.3. Use as the number of parallel workers for the children the maximum
of the value computed in step 1 and the value computed in step 2.
Ah, now that I closely look at compute_parallel_worker(), I see what
you are getting at.
For plain unpartitioned table, parallel_workers is calculated as
roughly equal to log(num_pages) (actually it is log3). So if the table
size is n, the workers will be log(n). So if it is partitioned into p
partitions of size n/p each, still the number of workers should be
log(n). Whereas, in the patch, it is calculated as (total of all the
child workers) i.e. n * log(n/p) for this case. But log(n) != p *
log(x/p). For e.g. log(1000) is much less than log(300) + log(300) +
log(300).
That means, the way it is calculated in the patch turns out to be much
larger than if it were calculated using log(total of sizes of all
children). So I think for the step 2 above, log(total_rel_size)
formula seems to be appropriate. What do you think ? For
compute_parallel_worker(), it is actually log3 by the way.
BTW this formula is just an extension of how parallel_workers is
calculated for an unpartitioned table.
For example, suppose that I have a scan of two children, one
of which has parallel_workers of 4, and the other of which has
parallel_workers of 3. If I pick parallel_workers of 7 for the
Parallel Append, that's probably too high. Had those two tables been
a single unpartitioned table, I would have picked 4 or 5 workers, not
7. On the other hand, if I pick parallel_workers of 4 or 5 for the
Parallel Append, and I finish with the larger table first, I think I
might as well throw all 4 of those workers at the smaller table even
though it would normally have only used 3 workers.Having the extra 1-2 workers exit does not seem better.
It is here, where I didn't understand exactly why would we want to
assign these extra workers to a subplan which tells use that it is
already being run by 'parallel_workers' number of workers.The decision to use fewer workers for a smaller scan isn't really
because we think that using more workers will cause a regression.
It's because we think it may not help very much, and because it's not
worth firing up a ton of workers for a relatively small scan given
that workers are a limited resource. I think once we've got a bunch
of workers started, we might as well try to use them.
One possible side-effect I see due to this is : Other sessions might
not get a fair share of workers due to this. But again, there might be
counter argument that, because Append is now focussing all the workers
on a last subplan, it may finish faster, and release *all* of its
workers earlier.
BTW, there is going to be some logic change in the choose-next-subplan
algorithm if we consider giving extra workers to subplans.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Feb 15, 2017 at 11:15 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
If the user is ready throw 200 workers and if the subplans can use
them to speed up the query 200 times (obviously I am exaggerating),
why not to use those? When the user set
max_parallel_workers_per_gather to that high a number, he meant it to
be used by a gather, and that's what we should be doing.
The reason is because of what Amit Khandekar wrote in his email -- you
get a result with a partitioned table that is wildly inconsistent with
the result you get for an unpartitioned table. You could equally well
argue that if the user sets max_parallel_workers_per_gather to 200,
and there's a parallel sequential scan of an 8MB table to be
performed, we ought to use all 200 workers for that. But the planner
in fact estimates a much lesser number of workers, because using 200
workers for that task wastes a lot of resources for no real
performance benefit. If you partition that 8MB table into 100 tables
that are each 80kB, that shouldn't radically increase the number of
workers that get used.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Feb 16, 2017 at 1:34 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
What I was thinking about is something like this:
1. First, take the maximum parallel_workers value from among all the children.
2. Second, compute log2(num_children)+1 and round up. So, for 1
child, 1; for 2 children, 2; for 3-4 children, 3; for 5-8 children, 4;
for 9-16 children, 5, and so on.3. Use as the number of parallel workers for the children the maximum
of the value computed in step 1 and the value computed in step 2.Ah, now that I closely look at compute_parallel_worker(), I see what
you are getting at.For plain unpartitioned table, parallel_workers is calculated as
roughly equal to log(num_pages) (actually it is log3). So if the table
size is n, the workers will be log(n). So if it is partitioned into p
partitions of size n/p each, still the number of workers should be
log(n). Whereas, in the patch, it is calculated as (total of all the
child workers) i.e. n * log(n/p) for this case. But log(n) != p *
log(x/p). For e.g. log(1000) is much less than log(300) + log(300) +
log(300).That means, the way it is calculated in the patch turns out to be much
larger than if it were calculated using log(total of sizes of all
children). So I think for the step 2 above, log(total_rel_size)
formula seems to be appropriate. What do you think ? For
compute_parallel_worker(), it is actually log3 by the way.BTW this formula is just an extension of how parallel_workers is
calculated for an unpartitioned table.
log(total_rel_size) would be a reasonable way to estimate workers when
we're scanning an inheritance hierarchy, but I'm hoping Parallel
Append is also going to apply to UNION ALL queries, where there's no
concept of the total rel size. For that we need something else, which
is why the algorithm that I proposed upthread doesn't rely on it.
The decision to use fewer workers for a smaller scan isn't really
because we think that using more workers will cause a regression.
It's because we think it may not help very much, and because it's not
worth firing up a ton of workers for a relatively small scan given
that workers are a limited resource. I think once we've got a bunch
of workers started, we might as well try to use them.One possible side-effect I see due to this is : Other sessions might
not get a fair share of workers due to this. But again, there might be
counter argument that, because Append is now focussing all the workers
on a last subplan, it may finish faster, and release *all* of its
workers earlier.
Right. I think in general it's pretty clear that there are possible
fairness problems with parallel query. The first process that comes
along seizes however many workers it thinks it should use, and
everybody else can use whatever (if anything) is left. In the long
run, I think it would be cool to have a system where workers can leave
one parallel query in progress and join a different one (or exit and
spawn a new worker to join a different one), automatically rebalancing
as the number of parallel queries in flight fluctuates. But that's
clearly way beyond anything we can do right now. I think we should
assume that any parallel workers our process has obtained are ours to
use for the duration of the query, and use them as best we can. Note
that even if the Parallel Append tells one of the workers that there
are no more tuples and it should go away, some higher level of the
query plan could make a different choice anyway; there might be
another Append elsewhere in the plan tree.
BTW, there is going to be some logic change in the choose-next-subplan
algorithm if we consider giving extra workers to subplans.
I'm not sure that it's going to be useful to make this logic very
complicated. I think the most important thing is to give 1 worker to
each plan before we give a second worker to any plan. In general I
think it's sufficient to assign a worker that becomes available to the
subplan with the fewest number of workers (or one of them, if there's
a tie) without worrying too much about the target number of workers
for that subplan.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Feb 16, 2017 at 8:15 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Feb 15, 2017 at 11:15 PM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:If the user is ready throw 200 workers and if the subplans can use
them to speed up the query 200 times (obviously I am exaggerating),
why not to use those? When the user set
max_parallel_workers_per_gather to that high a number, he meant it to
be used by a gather, and that's what we should be doing.The reason is because of what Amit Khandekar wrote in his email -- you
get a result with a partitioned table that is wildly inconsistent with
the result you get for an unpartitioned table. You could equally well
argue that if the user sets max_parallel_workers_per_gather to 200,
and there's a parallel sequential scan of an 8MB table to be
performed, we ought to use all 200 workers for that. But the planner
in fact estimates a much lesser number of workers, because using 200
workers for that task wastes a lot of resources for no real
performance benefit. If you partition that 8MB table into 100 tables
that are each 80kB, that shouldn't radically increase the number of
workers that get used.
That's true for a partitioned table, but not necessarily for every
append relation. Amit's patch is generic for all append relations. If
the child plans are joins or subquery segments of set operations, I
doubt if the same logic works. It may be better if we throw as many
workers (or some function "summing" those up) as specified by those
subplans. I guess, we have to use different logic for append relations
which are base relations and append relations which are not base
relations.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 16 February 2017 at 20:37, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Feb 16, 2017 at 1:34 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
What I was thinking about is something like this:
1. First, take the maximum parallel_workers value from among all the children.
2. Second, compute log2(num_children)+1 and round up. So, for 1
child, 1; for 2 children, 2; for 3-4 children, 3; for 5-8 children, 4;
for 9-16 children, 5, and so on.3. Use as the number of parallel workers for the children the maximum
of the value computed in step 1 and the value computed in step 2.Ah, now that I closely look at compute_parallel_worker(), I see what
you are getting at.For plain unpartitioned table, parallel_workers is calculated as
roughly equal to log(num_pages) (actually it is log3). So if the table
size is n, the workers will be log(n). So if it is partitioned into p
partitions of size n/p each, still the number of workers should be
log(n). Whereas, in the patch, it is calculated as (total of all the
child workers) i.e. n * log(n/p) for this case. But log(n) != p *
log(x/p). For e.g. log(1000) is much less than log(300) + log(300) +
log(300).That means, the way it is calculated in the patch turns out to be much
larger than if it were calculated using log(total of sizes of all
children). So I think for the step 2 above, log(total_rel_size)
formula seems to be appropriate. What do you think ? For
compute_parallel_worker(), it is actually log3 by the way.BTW this formula is just an extension of how parallel_workers is
calculated for an unpartitioned table.log(total_rel_size) would be a reasonable way to estimate workers when
we're scanning an inheritance hierarchy, but I'm hoping Parallel
Append is also going to apply to UNION ALL queries, where there's no
concept of the total rel size.
Yes ParallelAppend also gets used in UNION ALL.
For that we need something else, which
is why the algorithm that I proposed upthread doesn't rely on it.
The log2(num_children)+1 formula which you proposed does not take into
account the number of workers for each of the subplans, that's why I
am a bit more inclined to look for some other logic. May be, treat the
children as if they belong to partitions, and accordingly calculate
the final number of workers. So for 2 children with 4 and 5 workers
respectively, Append parallel_workers would be : log3(3^4 + 3^5) .
The decision to use fewer workers for a smaller scan isn't really
because we think that using more workers will cause a regression.
It's because we think it may not help very much, and because it's not
worth firing up a ton of workers for a relatively small scan given
that workers are a limited resource. I think once we've got a bunch
of workers started, we might as well try to use them.One possible side-effect I see due to this is : Other sessions might
not get a fair share of workers due to this. But again, there might be
counter argument that, because Append is now focussing all the workers
on a last subplan, it may finish faster, and release *all* of its
workers earlier.Right. I think in general it's pretty clear that there are possible
fairness problems with parallel query. The first process that comes
along seizes however many workers it thinks it should use, and
everybody else can use whatever (if anything) is left. In the long
run, I think it would be cool to have a system where workers can leave
one parallel query in progress and join a different one (or exit and
spawn a new worker to join a different one), automatically rebalancing
as the number of parallel queries in flight fluctuates. But that's
clearly way beyond anything we can do right now. I think we should
assume that any parallel workers our process has obtained are ours to
use for the duration of the query, and use them as best we can.
Note that even if the Parallel Append tells one of the workers that there
are no more tuples and it should go away, some higher level of the
query plan could make a different choice anyway; there might be
another Append elsewhere in the plan tree.
Yeah, that looks good enough to justify not losing the workers
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
Do we have any performance measurements where we see that Goal B
performs better than Goal A, in such a situation? Do we have any
performance measurement comparing these two approaches in other
situations. If implementation for Goal B beats that of Goal A always,
we can certainly implement it directly. But it may not.
I will get back with some performance numbers.
Also, separating patches for Goal A and Goal B might make reviews easier.
Do you anyways want the patch with the current state to be split ?
Right now, I am not sure how exactly you need me to split it.
BTW, Right now, the patch does not consider non-partial paths for a
child which has partial paths. Do we know, for sure, that a path
containing partial paths for a child, which has it, is always going to
be cheaper than the one which includes non-partial path. If not,
should we build another paths which contains non-partial paths for all
child relations. This sounds like a 0/1 knapsack problem.I didn't quite get this. We do create a non-partial Append path using
non-partial child paths anyways.Let's say a given child-relation has both partial and non-partial
paths, your approach would always pick up a partial path. But now that
parallel append can handle non-partial paths as well, it may happen
that picking up non-partial path instead of partial one when both are
available gives an overall better performance. Have we ruled out that
possibility.
Yes, one Append can contain a child c1 with partial path, another
Append path can contain child c1 with non-partial path, and each of
this combination can have two more combinations for child2, and so on,
leading to too many Append paths. I think that's what you referred to
as 0/1 knapsack problem. Right, this does not seem worth it.
I had earlier considered adding a partial Append path containing only
non-partial paths, but for some reason I had concluded that it's not
worth having this path, as it's cost is most likely going to be higher
due to presence of all single-worker paths *and* also a Gather above
them. I should have documented the reason. Let me give a thought on
this.
Let me try keeping the per-subplan max_worker info in Append path
itself, like I mentioned above. If that works, the bitmap will be
replaced by max_worker field. In case of non-partial subpath,
max_worker will be 1. (this is the same info kept in AppendState node
in the patch, but now we might need to keep it in Append path node as
well).It will be better if we can fetch that information from each subpath
when creating the plan. As I have explained before, a path is minimal
structure, which should be easily disposable, when throwing away the
path.Now in the v2 patch, we store per-subplan worker count. But still, we
cannot use the path->parallel_workers to determine whether it's a
partial path. This is because even for a non-partial path, it seems
the parallel_workers can be non-zero. For e.g., in
create_subqueryscan_path(), it sets path->parallel_workers to
subpath->parallel_workers. But this path is added as a non-partial
path. So we need a separate info as to which of the subpaths in Append
path are partial subpaths. So in the v2 patch, I continued to use
Bitmapset in AppendPath. But in Append plan node, number of workers is
calculated using this bitmapset. Check the new function
get_append_num_workers().If the subpath from childrel->partial_pathlist, then we set the
corresponding bit in the bitmap. Now we can infer that for any path if
that path is found in path->parent->partial_pathlist. Since the code
always chooses the first partial path, the search in partial_pathlist
should not affect performance. So, we can avoid maintaining a bitmap
in the path and keep accumulating it when collapsing append paths.
Thanks. Accordingly did these changes in attached v4 patch.
get_append_num_workers() now uses
linitial(path->parent->partial_pathlist) to determine whether the
subpath is a partial or a non-partial path. Removed the bitmapset
field from AppendPath.
12. cost_append() essentially adds costs of all the subpaths and then divides
by parallel_divisor. This might work if all the subpaths are partial paths. But
for the subpaths which are not partial, a single worker will incur the whole
cost of that subpath. Hence just dividing all the total cost doesn't seem the
right thing to do. We should apply different logic for costing non-partial
subpaths and partial subpaths.WIth the current partial path costing infrastructure, it is assumed
that a partial path node should return the average per-worker cost.
Hence, I thought it would be best to do it in a similar way for
Append. But let me think if we can do something. With the current
parallelism costing infrastructure, I am not sure though.The current parallel mechanism is in sync with that costing. Each
worker is supposed to take the same burden, hence the same (average)
cost. But it will change when a single worker has to scan an entire
child relation and different child relations have different sizes.I gave more thought on this. Considering each subplan has different
number of workers, I think it makes sense to calculate average
per-worker cost even in parallel Append. In case of non-partial
subplan, a single worker will execute it, but it will next choose
another subplan. So on average each worker is going to process the
same number of rows, and also the same amount of CPU. And that amount
of CPU cost and rows cost should be calculated by taking the total
count and dividing it by number of workers (parallel_divsor actually).That's not entirely true. Consider N child relations with chosen paths
with costs C1, C2, ... CN which are very very different. If there are
N workers, the total cost should correspond to the highest of the
costs of subpaths, since no worker will execute more than one plan.
The unfortunate worker which executes the costliest path would take
the longest time.
Yeah, there seems to be no specific method that can compute the total
cost as the maximum of all the subplans total cost. So the assumption
is that there would be roughly equal distribution of workers.
In the new patch, there is a new test case output modification for
inherit.sql , because that test case started failing on account of
getting a ParallelAppend plan instead of Merge Append for an
inheritence table where seqscan was disabled.
Attachments:
ParallelAppend_v4.patchapplication/octet-stream; name=ParallelAppend_v4.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 98d4f1e..6b34dab 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeSeqscan.h"
@@ -206,6 +207,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -258,6 +263,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecIndexScanInitializeDSM((IndexScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
@@ -737,6 +746,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
case T_IndexScanState:
ExecIndexScanInitializeWorker((IndexScanState *) planstate, toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_ForeignScanState:
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..97bfc89 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,48 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendInfo
+{
+ int pa_num_workers; /* workers currently executing the subplan */
+ int pa_max_workers; /* max workers that should run the subplan */
+} ParallelAppendInfo;
+
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +116,22 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ return parallel_append_next(appendstate);
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +233,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +252,14 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished",
+ MyProcPid);
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +281,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +336,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +351,223 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+ List *num_workers_list = ((Append*)node->ps.plan)->num_workers;
+ ListCell *lc;
+ int i;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ Assert(node->as_nplans == list_length(num_workers_list));
+
+ i = 0;
+ foreach(lc, num_workers_list)
+ {
+ /* Initialize the max workers count for each subplan. */
+ padesc->pa_info[i].pa_max_workers = lfirst_int(lc);
+
+ /*
+ * Also, initialize current number of workers. Just setting all the
+ * number of workers to 0 is enough. The logic of choosing the next
+ * plan in workers will take care of initializing everything else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+
+ i++;
+ }
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ elog(DEBUG2, "Parallelappend : pid %d : finishing plan %d",
+ MyProcPid, whichplan);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
+
+ /* Ignore plans that are already done processing */
+ if (painfo->pa_num_workers == -1)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d"
+ " since pa_num_workers is -1",
+ MyProcPid, whichplan);
+ continue;
+ }
+
+ /* Ignore plans that are already being processed by max_workers */
+ if (painfo->pa_num_workers == painfo->pa_max_workers)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : ignoring plan %d,"
+ " since reached max_worker count %d",
+ MyProcPid, whichplan, painfo->pa_max_workers);
+ continue;
+ }
+
+ /*
+ * Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
+ }
+
+ /* Increment worker count for the chosen node, if at all we found one. */
+ if (min_whichplan != PA_INVALID_PLAN)
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+ elog(DEBUG2, "ParallelAppend : pid %d : Chosen plan : %d",
+ MyProcPid, min_whichplan);
+
+ /*
+ * If we didn't find any node to work on, it means each subplan is either
+ * finished or has reached it's pa_max_workers. In such case, should this
+ * worker wait for some subplan to have its worker count drop below its
+ * pa_max_workers so that it can choose that subplan ? It turns out that
+ * it's not worth again finding a subplan to work on. Non-partial subplan
+ * anyway can have only one worker, and that worker will execute it to
+ * completion. For a partial subplan, if at all it reaches pa_max_workers,
+ * it's worker count will reduce only when it's workers find that there is
+ * nothing more to be executed, so there is no point taking up such node if
+ * it's worker count reduces. In conclusion, just stop executing once we
+ * don't find nodes to work on. Indicate the same by returning false.
+ */
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 05d8538..160860e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_NODE_FIELD(num_workers);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..a007f69 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_NODE_FIELD(num_workers);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d2f69fe..6142488 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1539,6 +1539,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_NODE_FIELD(num_workers);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index eeacf81..648d04a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1232,14 +1232,50 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
+ }
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
+ Path *parallel_safe_path = NULL;
+ foreach(lcp, childrel->pathlist)
+ {
+ Path *child_path = (Path *) lfirst(lcp);
+ if (child_path->parallel_safe &&
+ child_path->param_info == NULL)
+ {
+ parallel_safe_path = child_path;
+ break;
+ }
+ }
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1322,24 +1358,10 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
-
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
+ int parallel_workers;
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths, NULL);
- /* Generate a partial append path. */
appendpath = create_append_path(rel, partial_subpaths, NULL,
parallel_workers);
add_partial_path(rel, (Path *) appendpath);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index d01630f..0ac1feb 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool enable_nestloop = true;
bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1560,6 +1561,82 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, Relids required_outer)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+
+ path->parallel_safe = path->parallel_safe &&
+ subpath->parallel_safe;
+
+ /* All child paths must have same parameterization */
+ Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 997bdcf..d36808b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/placeholder.h"
#include "optimizer/plancat.h"
@@ -194,7 +195,7 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, List *num_workers, List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -962,6 +963,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
Append *plan;
List *tlist = build_path_tlist(root, &best_path->path);
List *subplans = NIL;
+ List *num_workers_list;
ListCell *subpaths;
/*
@@ -1000,6 +1002,10 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
subplans = lappend(subplans, subplan);
}
+ /* Get a list of number of workers for each of the subplans */
+ (void) get_append_num_workers(best_path->subpaths,
+ &num_workers_list);
+
/*
* XXX ideally, if there's just one child, we'd not bother to generate an
* Append node but just return the single child. At the moment this does
@@ -1007,7 +1013,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, num_workers_list, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5009,7 +5015,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, List *num_workers, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5019,6 +5025,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->num_workers = num_workers;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 3d33d46..6f4ab23 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3334,10 +3334,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..7c244d9 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1192,6 +1192,55 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ * Optionally return the list of per-subplan worker count through
+ * 'per_subplan_workers'
+ */
+int
+get_append_num_workers(List *subpaths, List **per_subplan_workers)
+{
+ ListCell *lc;
+ int total_workers = 0;
+ int subplan_workers;
+ int i = 0;
+
+ if (per_subplan_workers)
+ *per_subplan_workers = NIL;
+
+ foreach(lc, subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ RelOptInfo *rel = subpath->parent;
+
+ /*
+ * If this subpath is actually the cheapest partial path, take into
+ * account its parallel workers, else consider one worker since it's
+ * non-partial.
+ */
+ if (rel->partial_pathlist != NIL &&
+ (Path *) linitial(rel->partial_pathlist) == subpath)
+ subplan_workers = subpath->parallel_workers;
+ else
+ subplan_workers = 1;
+
+ if (per_subplan_workers)
+ {
+ *per_subplan_workers =
+ lappend_int(*per_subplan_workers, subplan_workers);
+ }
+ total_workers += subplan_workers;
+ i++;
+ }
+
+ /* In no case use more than max_parallel_workers_per_gather. */
+ total_workers = Min(total_workers,
+ max_parallel_workers_per_gather);
+
+ return total_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1203,46 +1252,21 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
- ListCell *l;
pathnode->path.pathtype = T_Append;
pathnode->path.parent = rel;
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- foreach(l, subpaths)
- {
- Path *subpath = (Path *) lfirst(l);
-
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
-
- /* All child paths must have same parameterization */
- Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
- }
+ cost_append(&pathnode->path, subpaths, required_outer);
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 5d8fb2e..0efb5cf 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -901,6 +901,16 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
+
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 9f41bab..df5cfd5 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1184,12 +1185,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..8c06ee0 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ List *num_workers;
} Append;
/* ----------------
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 72200fa..1c47b4b 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -98,6 +99,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths, Relids required_outer);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..d5429c8 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -62,8 +63,10 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *subpaths, List **per_subplan_workers);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Relids required_outer,
+ int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index a8c8b28..1ef8c4d 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1337,6 +1337,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1403,6 +1404,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 48fb80e..75c6103 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..7a303fa 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(11 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(12 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index a8b7eb1..e9203a1 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -462,11 +462,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
On 16 February 2017 at 20:37, Robert Haas <robertmhaas@gmail.com> wrote:
I'm not sure that it's going to be useful to make this logic very
complicated. I think the most important thing is to give 1 worker to
each plan before we give a second worker to any plan. In general I
think it's sufficient to assign a worker that becomes available to the
subplan with the fewest number of workers (or one of them, if there's
a tie)
without worrying too much about the target number of workers for that subplan.
The reason I have considered per-subplan workers is , for instance, so
that we can respect the parallel_workers reloption set by the user for
different tables. Or for e.g., subquery1 is a big hash join needing
more workers, and subquery2 is a small table requiring quite lesser
workers, it seems to make sense to give more workers to subquery1.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Feb 17, 2017 at 11:44 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
That's true for a partitioned table, but not necessarily for every
append relation. Amit's patch is generic for all append relations. If
the child plans are joins or subquery segments of set operations, I
doubt if the same logic works. It may be better if we throw as many
workers (or some function "summing" those up) as specified by those
subplans. I guess, we have to use different logic for append relations
which are base relations and append relations which are not base
relations.
Well, I for one do not believe that if somebody writes a UNION ALL
with 100 branches, they should get 100 (or 99) workers. Generally
speaking, the sweet spot for parallel workers on queries we've tested
so far has been between 1 and 4. It's straining credulity to believe
that the number that's correct for parallel append is more than an
order of magnitude larger. Since increasing resource commitment by
the logarithm of the problem size has worked reasonably well for table
scans, I believe we should pursue a similar approach here. I'm
willing to negotiate on the details of what the formula I looked like,
but I'm not going to commit something that lets an Append relation try
to grab massively more resources than we'd use for some other plan
shape.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Feb 17, 2017 at 2:56 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
The log2(num_children)+1 formula which you proposed does not take into
account the number of workers for each of the subplans, that's why I
am a bit more inclined to look for some other logic. May be, treat the
children as if they belong to partitions, and accordingly calculate
the final number of workers. So for 2 children with 4 and 5 workers
respectively, Append parallel_workers would be : log3(3^4 + 3^5) .
In general this will give an answer not different by more than 1 or 2
from my answer, and often exactly the same. In the case you mention,
whether we get the same answer depends on which way you round:
log3(3^4+3^5) is 5 if you round down, 6 if you round up.
My formula is more aggressive when there are many subplans that are
not parallel or take only 1 worker, because I'll always use at least 5
workers for an append that has 9-16 children, whereas you might use
only 2 if you do log3(3^0+3^0+3^0+3^0+3^0+3^0+3^0+3^0+3^0). In that
case I like my formula better. With lots of separate children, the
chances of being able to use as many as 5 workers seem good. (Note
that using 9 workers as Ashutosh seems to be proposing would be a
waste if the different children have very unequal execution times,
because the workers that run children with short execution times can
be reused to run additional subplans while the long ones are still
running. Running a separate worker for each child only works out if
the shortest runtime is more than 50% of the longest runtime, which
may sometimes be true but doesn't seem like a good bet in general.)
Your formula is more aggressive when you have 3 children that all use
the same number of workers; it'll always decide on <number of workers
per child>+1, whereas mine won't add the extra worker in that case.
Possibly your formula is better than mine in that case, but I'm not
sure. If you have as many as 9 children that all want N workers, your
formula will decide on N+2 workers, but since my formula guarantees a
minimum of 5 workers in such cases, I'll probably be within 1 of
whatever answer you were getting.
Basically, I don't believe that the log3(n) thing is anything very
special or magical. The fact that I settled on that formula for
parallel sequential scan doesn't mean that it's exactly right for
every other case. I do think it's likely that increasing workers
logarithmically is a fairly decent strategy here, but I wouldn't get
hung up on using log3(n) in every case or making all of the answers
100% consistent according to some grand principal. I'm not even sure
log3(n) is right for parallel sequential scan, so insisting that
Parallel Append has to work that way when I had no better reason than
gut instinct for picking that for Parallel Sequential Scan seems to me
to be a little unprincipled. We're still in the early stages of this
parallel query experiment, and a decent number of these algorithms are
likely to change as we get more sophisticated. For now at least, it's
more important to pick things that work well pragmatically than to be
theoretically optimal.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sun, Feb 19, 2017 at 2:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Feb 17, 2017 at 11:44 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:That's true for a partitioned table, but not necessarily for every
append relation. Amit's patch is generic for all append relations. If
the child plans are joins or subquery segments of set operations, I
doubt if the same logic works. It may be better if we throw as many
workers (or some function "summing" those up) as specified by those
subplans. I guess, we have to use different logic for append relations
which are base relations and append relations which are not base
relations.Well, I for one do not believe that if somebody writes a UNION ALL
with 100 branches, they should get 100 (or 99) workers. Generally
speaking, the sweet spot for parallel workers on queries we've tested
so far has been between 1 and 4. It's straining credulity to believe
that the number that's correct for parallel append is more than an
order of magnitude larger. Since increasing resource commitment by
the logarithm of the problem size has worked reasonably well for table
scans, I believe we should pursue a similar approach here.
Thanks for that explanation. I makes sense. So, something like this
would work: total number of workers = some function of log(sum of
sizes of relations). The number of workers allotted to each segment
are restricted to the the number of workers chosen by the planner
while planning that segment. The patch takes care of the limit right
now. It needs to incorporate the calculation for total number of
workers for append.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Feb 20, 2017 at 10:54 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Sun, Feb 19, 2017 at 2:33 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Feb 17, 2017 at 11:44 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:That's true for a partitioned table, but not necessarily for every
append relation. Amit's patch is generic for all append relations. If
the child plans are joins or subquery segments of set operations, I
doubt if the same logic works. It may be better if we throw as many
workers (or some function "summing" those up) as specified by those
subplans. I guess, we have to use different logic for append relations
which are base relations and append relations which are not base
relations.Well, I for one do not believe that if somebody writes a UNION ALL
with 100 branches, they should get 100 (or 99) workers. Generally
speaking, the sweet spot for parallel workers on queries we've tested
so far has been between 1 and 4. It's straining credulity to believe
that the number that's correct for parallel append is more than an
order of magnitude larger. Since increasing resource commitment by
the logarithm of the problem size has worked reasonably well for table
scans, I believe we should pursue a similar approach here.Thanks for that explanation. I makes sense. So, something like this
would work: total number of workers = some function of log(sum of
sizes of relations). The number of workers allotted to each segment
are restricted to the the number of workers chosen by the planner
while planning that segment. The patch takes care of the limit right
now. It needs to incorporate the calculation for total number of
workers for append.
log(sum of sizes of relations) isn't well-defined for a UNION ALL query.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 19 February 2017 at 14:59, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Feb 17, 2017 at 2:56 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
The log2(num_children)+1 formula which you proposed does not take into
account the number of workers for each of the subplans, that's why I
am a bit more inclined to look for some other logic. May be, treat the
children as if they belong to partitions, and accordingly calculate
the final number of workers. So for 2 children with 4 and 5 workers
respectively, Append parallel_workers would be : log3(3^4 + 3^5) .In general this will give an answer not different by more than 1 or 2
from my answer, and often exactly the same. In the case you mention,
whether we get the same answer depends on which way you round:
log3(3^4+3^5) is 5 if you round down, 6 if you round up.My formula is more aggressive when there are many subplans that are
not parallel or take only 1 worker, because I'll always use at least 5
workers for an append that has 9-16 children, whereas you might use
only 2 if you do log3(3^0+3^0+3^0+3^0+3^0+3^0+3^0+3^0+3^0). In that
case I like my formula better. With lots of separate children, the
chances of being able to use as many as 5 workers seem good. (Note
that using 9 workers as Ashutosh seems to be proposing would be a
waste if the different children have very unequal execution times,
because the workers that run children with short execution times can
be reused to run additional subplans while the long ones are still
running. Running a separate worker for each child only works out if
the shortest runtime is more than 50% of the longest runtime, which
may sometimes be true but doesn't seem like a good bet in general.)Your formula is more aggressive when you have 3 children that all use
the same number of workers; it'll always decide on <number of workers
per child>+1, whereas mine won't add the extra worker in that case.
Possibly your formula is better than mine in that case, but I'm not
sure. If you have as many as 9 children that all want N workers, your
formula will decide on N+2 workers, but since my formula guarantees a
minimum of 5 workers in such cases, I'll probably be within 1 of
whatever answer you were getting.
Yeah, that seems to be right in most of the cases. The only cases
where your formula seems to give too few workers is for something like
: (2, 8, 8). For such subplans, we should at least allocate 8 workers.
It turns out that in most of the cases in my formula, the Append
workers allocated is just 1 worker more than the max per-subplan
worker count. So in (2, 1, 1, 8), it will be a fraction more than 8.
So in the patch, in addition to the log2() formula you proposed, I
have made sure that it allocates at least equal to max(per-subplan
parallel_workers values).
BTW, there is going to be some logic change in the choose-next-subplan
algorithm if we consider giving extra workers to subplans.I'm not sure that it's going to be useful to make this logic very
complicated. I think the most important thing is to give 1 worker to
each plan before we give a second worker to any plan. In general I
think it's sufficient to assign a worker that becomes available to the
subplan with the fewest number of workers (or one of them, if there's
a tie) without worrying too much about the target number of workers
for that subplan.
In the attached v5 patch, the logic of distributing the workers is now
kept simple : it just distributes the workers equally without
considering the per-sublan parallel_workers value. I have retained the
earlier logic of choosing the plan with minimum current workers. But
now that the pa_max_workers is not needed, I removed it, and instead a
partial_plans bitmapset is added in the Append node. Once a worker
picks up a non-partial subplan, it immediately changes its
pa_num_workers to -1. Whereas for partial subplans, the worker sets it
to -1 only after it finishes executing it.
Effectively, in parallel_append_next(), the check for whether subplan
is executing with max parallel_workers is now removed, and all code
that was using pa_max_workers is now removed.
Ashutosh Bapat <ashutosh.bapat@enterprisedb.com> wrote:
10. We should probably move the parallel_safe calculation out of cost_append(). + path->parallel_safe = path->parallel_safe && + subpath->parallel_safe;11. This check shouldn't be part of cost_append(). + /* All child paths must have same parameterization */ + Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
Moved out these two statements from cost_append(). Did it separately
in create_append_path().
Also, I have removed some elog() statements which were there while
inside Spinlock in parallel_append_next().
On 17 January 2017 at 11:10, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:
I was looking at the executor portion of this patch and I noticed that in
exec_append_initialize_next():if (appendstate->as_padesc)
return parallel_append_next(appendstate);/*
* Not parallel-aware. Fine, just go on to the next subplan in the
* appropriate direction.
*/
if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
appendstate->as_whichplan++;
else
appendstate->as_whichplan--;which seems to mean that executing Append in parallel mode disregards the
scan direction. I am not immediately sure what implications that has, so
I checked what heap scan does when executing in parallel mode, and found
this in heapgettup():else if (backward)
{
/* backward parallel scan not supported */
Assert(scan->rs_parallel == NULL);Perhaps, AppendState.as_padesc would not have been set if scan direction
is backward, because parallel mode would be disabled for the whole query
in that case (PlannerGlobal.parallelModeOK = false). Maybe add an
Assert() similar to one in heapgettup().
Right. Thanks for noticing this. I have added a similar Assert in
exec_append_initialize_next().
Attachments:
ParallelAppend_v5.patchapplication/octet-stream; name=ParallelAppend_v5.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index de0e2ba..6357f29 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeSeqscan.h"
@@ -213,6 +214,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -273,6 +278,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -771,6 +780,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..a5ffb38 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,56 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendInfo
+{
+ /*
+ * pa_num_workers : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_num_workers to -1, so that no new
+ * worker pick this subplan. For non-partial subplan, a worker which picks
+ * up that subplan, it should immediately set to -1, so as to make sure
+ * there are no more than 1 worker assigned to this subplan. In general, -1
+ * means workers should stop picking it.
+ */
+ int pa_num_workers;
+
+} ParallelAppendInfo;
+
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +124,27 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ {
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(!ScanDirectionIsBackward(appendstate->ps.state->es_direction));
+
+ return parallel_append_next(appendstate);
+ }
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +246,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +265,14 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ {
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished",
+ MyProcPid);
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +294,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +349,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +364,204 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
+ }
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+ int i;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ for (i = 0; i < node->as_nplans; i++)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ elog(DEBUG2, "Parallelappend : pid %d : finishing plan %d",
+ MyProcPid, whichplan);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
+
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (painfo->pa_num_workers == -1)
+ continue;
+
+ /*
+ * Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+
+ /*
+ * Increment worker count for the chosen node, if at all we found one.
+ * For non-partial plans, set it to -1 instead, so that no other workers
+ * run it.
+ */
+ if (min_whichplan != PA_INVALID_PLAN)
+ {
+ if (bms_is_member(min_whichplan,
+ ((Append*)state->ps.plan)->partial_subplans_set))
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+ else
+ padesc->pa_info[min_whichplan].pa_num_workers = -1;
+ }
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+ elog(DEBUG2, "ParallelAppend : pid %d : Chosen plan : %d",
+ MyProcPid, min_whichplan);
+
+ /*
+ * If we didn't find any node to work on, stop executing. Indicate the same
+ * by returning false.
+ */
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index bb2a8a3..67f722a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_BITMAPSET_FIELD(partial_subplans_set);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index b3802b4..69f1139 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_BITMAPSET_FIELD(partial_subplans_set);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 05bf2e9..6d3ca5d 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1537,6 +1537,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_BITMAPSET_FIELD(partial_subplans_set);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 87a3faf..7a59c8e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -1232,14 +1232,50 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
+ }
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
+ Path *parallel_safe_path = NULL;
+ foreach(lcp, childrel->pathlist)
+ {
+ Path *child_path = (Path *) lfirst(lcp);
+ if (child_path->parallel_safe &&
+ child_path->param_info == NULL)
+ {
+ parallel_safe_path = child_path;
+ break;
+ }
+ }
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1322,24 +1358,10 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
-
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
+ int parallel_workers;
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths);
- /* Generate a partial append path. */
appendpath = create_append_path(rel, partial_subpaths, NULL,
parallel_workers);
add_partial_path(rel, (Path *) appendpath);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c138f57..ccd6733 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -126,6 +126,7 @@ bool enable_nestloop = true;
bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1559,6 +1560,70 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 1e953b4..04b0414 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -29,6 +29,7 @@
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
#include "optimizer/paths.h"
#include "optimizer/placeholder.h"
#include "optimizer/plancat.h"
@@ -194,7 +195,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, Bitmapset *partial_plans_set,
+ List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -962,6 +964,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
List *tlist = build_path_tlist(root, &best_path->path);
List *subplans = NIL;
ListCell *subpaths;
+ Bitmapset *partial_subplans_set;
+ int i;
/*
* The subpaths list could be empty, if every child was proven empty by
@@ -987,12 +991,25 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
return plan;
}
- /* Build the plan for each child */
+ /* Build the plan for each child, and a bitmapset of partial subpaths */
+ partial_subplans_set = NULL;
+ i = 0;
foreach(subpaths, best_path->subpaths)
{
Path *subpath = (Path *) lfirst(subpaths);
+ RelOptInfo *rel = subpath->parent;
Plan *subplan;
+ /*
+ * If this subpath is actually the cheapest partial path, add this into
+ * the partial path set.
+ */
+ if (rel->partial_pathlist != NIL &&
+ (Path *) linitial(rel->partial_pathlist) == subpath)
+ partial_subplans_set = bms_add_member(partial_subplans_set, i);
+
+ i++;
+
/* Must insist that all children return the same tlist */
subplan = create_plan_recurse(root, subpath, CP_EXACT_TLIST);
@@ -1006,7 +1023,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, partial_subplans_set, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5003,7 +5020,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, Bitmapset *partial_plans_set, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5013,6 +5030,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->partial_subplans_set = partial_plans_set;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ca0ae78..fb91264 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3334,10 +3334,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3248296..1b8e362 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1192,6 +1192,67 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) i.e. ln(num_subpaths) / ln(2) */
+ log2w = log(list_length(subpaths)) / 0.693 ;
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1210,40 +1271,27 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, subpaths);
+
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index f8b073d..2994413 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -902,6 +902,16 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
+
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6332ea0..c887be6 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1185,12 +1186,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f72f7a8..6d772ca 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ Bitmapset *partial_subplans_set;
} Append;
/* ----------------
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 72200fa..484e179 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -66,6 +66,7 @@ extern bool enable_nestloop;
extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -98,6 +99,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 53cad24..dbec534 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -62,8 +63,10 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Relids required_outer,
+ int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 795d9f5..367d23f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1337,6 +1337,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1403,6 +1404,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 75558d0..3071bae 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index d48abd7..7a303fa 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,20 +70,21 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(11 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(12 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index 836ec22..0636f08 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -462,11 +462,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
On Wed, Mar 8, 2017 at 2:00 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Yeah, that seems to be right in most of the cases. The only cases
where your formula seems to give too few workers is for something like
: (2, 8, 8). For such subplans, we should at least allocate 8 workers.
It turns out that in most of the cases in my formula, the Append
workers allocated is just 1 worker more than the max per-subplan
worker count. So in (2, 1, 1, 8), it will be a fraction more than 8.
So in the patch, in addition to the log2() formula you proposed, I
have made sure that it allocates at least equal to max(per-subplan
parallel_workers values).
Yeah, I agree with that.
Some review:
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose
next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
Instead of having ParallelAppendInfo, how about just int
pa_workers[FLEXIBLE_ARRAY_MEMBER]? The second structure seems like
overkill, at least for now.
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
I don't think this is buying you anything, and suggest backing it out.
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(!ScanDirectionIsBackward(appendstate->ps.state->es_direction));
I think you could assert ScanDirectionIsForward, couldn't you?
NoMovement, I assume, is right out.
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already
finished",
+ MyProcPid);
Please remove (and all similar cases also).
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
I'd use the type name instead.
+ for (i = 0; i < node->as_nplans; i++)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
Here I'd use memset.
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
Maybe just return (min_whichplan != PA_INVALID_PLAN);
- childrel->cheapest_total_path);
+
childrel->cheapest_total_path);
Unnecessary.
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
+ }
Don't need to add braces.
+ /*
+ * Extract the first unparameterized, parallel-safe one among the
+ * child paths.
+ */
Can we use get_cheapest_parallel_safe_total_inner for this, from
a71f10189dc10a2fe422158a2c9409e0f77c6b9e?
+ if (rel->partial_pathlist != NIL &&
+ (Path *) linitial(rel->partial_pathlist) == subpath)
+ partial_subplans_set = bms_add_member(partial_subplans_set, i);
This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason? I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, 0);
Unnecessary.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
+ if (rel->partial_pathlist != NIL && + (Path *) linitial(rel->partial_pathlist) == subpath) + partial_subplans_set = bms_add_member(partial_subplans_set, i);This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason? I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.
I agree that assuming that a subpath is non-partial path if it's not
cheapest of the partial paths is risky. In fact, we can not assume
that even when it's not one of the partial_paths since it could have
been kicked out or was never added to the partial path list like
reparameterized path. But if we have to save the information about
which of the subpaths are partial paths and which are not in
AppendPath, it would take some memory, noticeable for thousands of
partitions, which will leak if the path doesn't make into the
rel->pathlist. The purpose of that information is to make sure that we
allocate only one worker to that plan. I suggested that we use
path->parallel_workers for the same, but it seems that's not
guaranteed to be reliable. The reasons were discussed upthread. Is
there any way to infer whether we can allocate more than one workers
to a plan by looking at the corresponding path?
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 9, 2017 at 7:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
+ if (rel->partial_pathlist != NIL && + (Path *) linitial(rel->partial_pathlist) == subpath) + partial_subplans_set = bms_add_member(partial_subplans_set, i);This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason? I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.I agree that assuming that a subpath is non-partial path if it's not
cheapest of the partial paths is risky. In fact, we can not assume
that even when it's not one of the partial_paths since it could have
been kicked out or was never added to the partial path list like
reparameterized path. But if we have to save the information about
which of the subpaths are partial paths and which are not in
AppendPath, it would take some memory, noticeable for thousands of
partitions, which will leak if the path doesn't make into the
rel->pathlist.
True, but that's no different from the situation for any other Path
node that has substructure. For example, an IndexPath has no fewer
than 5 list pointers in it. Generally we assume that the number of
paths won't be large enough for the memory used to really matter, and
I think that will also be true here. And an AppendPath has a list of
subpaths, and if I'm not mistaken, those list nodes consume more
memory than the tracking information we're thinking about here will.
I think you're thinking about this issue because you've been working
on partitionwise join where memory consumption is a big issue, but
there are a lot of cases where that isn't really a big deal.
The purpose of that information is to make sure that we
allocate only one worker to that plan. I suggested that we use
path->parallel_workers for the same, but it seems that's not
guaranteed to be reliable. The reasons were discussed upthread. Is
there any way to infer whether we can allocate more than one workers
to a plan by looking at the corresponding path?
I think it would be smarter to track it some other way. Either keep
two lists of paths, one of which is the partial paths and the other of
which is the parallel-safe paths, or keep a bitmapset indicating which
paths fall into which category. I am not going to say there's no way
we could make it work without either of those things -- looking at the
parallel_workers flag might be made to work, for example -- but the
design idea I had in mind when I put this stuff into place was that
you keep them separate in other ways, not by the data they store
inside them. I think it will be more robust if we keep to that
principle.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 9, 2017 at 6:28 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Mar 9, 2017 at 7:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:+ if (rel->partial_pathlist != NIL && + (Path *) linitial(rel->partial_pathlist) == subpath) + partial_subplans_set = bms_add_member(partial_subplans_set, i);This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason? I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.I agree that assuming that a subpath is non-partial path if it's not
cheapest of the partial paths is risky. In fact, we can not assume
that even when it's not one of the partial_paths since it could have
been kicked out or was never added to the partial path list like
reparameterized path. But if we have to save the information about
which of the subpaths are partial paths and which are not in
AppendPath, it would take some memory, noticeable for thousands of
partitions, which will leak if the path doesn't make into the
rel->pathlist.True, but that's no different from the situation for any other Path
node that has substructure. For example, an IndexPath has no fewer
than 5 list pointers in it. Generally we assume that the number of
paths won't be large enough for the memory used to really matter, and
I think that will also be true here. And an AppendPath has a list of
subpaths, and if I'm not mistaken, those list nodes consume more
memory than the tracking information we're thinking about here will.
What I have observed is that we try to keep the memory usage to a
minimum, trying to avoid memory consumption as much as possible. Most
of that substructure gets absorbed by the planner or is shared across
paths. Append path lists are an exception to that, but we need
something to hold all subpaths together and list is PostgreSQL's way
of doing it. So, that's kind of unavoidable. And may be we will find
some reason for almost every substructure in paths.
I think you're thinking about this issue because you've been working
on partitionwise join where memory consumption is a big issue, but
there are a lot of cases where that isn't really a big deal.
:).
The purpose of that information is to make sure that we
allocate only one worker to that plan. I suggested that we use
path->parallel_workers for the same, but it seems that's not
guaranteed to be reliable. The reasons were discussed upthread. Is
there any way to infer whether we can allocate more than one workers
to a plan by looking at the corresponding path?I think it would be smarter to track it some other way. Either keep
two lists of paths, one of which is the partial paths and the other of
which is the parallel-safe paths, or keep a bitmapset indicating which
paths fall into which category.
I like two lists: it consumes almost no memory (two list headers
instead of one) compared to non-parallel-append when there are
non-partial paths and what more, it consumes no extra memory when all
paths are partial.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10 March 2017 at 10:13, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Thu, Mar 9, 2017 at 6:28 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Mar 9, 2017 at 7:42 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:+ if (rel->partial_pathlist != NIL && + (Path *) linitial(rel->partial_pathlist) == subpath) + partial_subplans_set = bms_add_member(partial_subplans_set, i);This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason?
Yes, there was an assumption that append subpath will be either a
cheapest non-partial path, or the cheapest (i.e. first in the list)
partial path, although in the patch there is no Asserts to make sure
that a common rule has been followed at both these places.
I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.I agree that assuming that a subpath is non-partial path if it's not
cheapest of the partial paths is risky. In fact, we can not assume
that even when it's not one of the partial_paths since it could have
been kicked out or was never added to the partial path list like
reparameterized path. But if we have to save the information about
which of the subpaths are partial paths and which are not in
AppendPath, it would take some memory, noticeable for thousands of
partitions, which will leak if the path doesn't make into the
rel->pathlist.True, but that's no different from the situation for any other Path
node that has substructure. For example, an IndexPath has no fewer
than 5 list pointers in it. Generally we assume that the number of
paths won't be large enough for the memory used to really matter, and
I think that will also be true here. And an AppendPath has a list of
subpaths, and if I'm not mistaken, those list nodes consume more
memory than the tracking information we're thinking about here will.What I have observed is that we try to keep the memory usage to a
minimum, trying to avoid memory consumption as much as possible. Most
of that substructure gets absorbed by the planner or is shared across
paths. Append path lists are an exception to that, but we need
something to hold all subpaths together and list is PostgreSQL's way
of doing it. So, that's kind of unavoidable. And may be we will find
some reason for almost every substructure in paths.I think you're thinking about this issue because you've been working
on partitionwise join where memory consumption is a big issue, but
there are a lot of cases where that isn't really a big deal.:).
The purpose of that information is to make sure that we
allocate only one worker to that plan. I suggested that we use
path->parallel_workers for the same, but it seems that's not
guaranteed to be reliable. The reasons were discussed upthread. Is
there any way to infer whether we can allocate more than one workers
to a plan by looking at the corresponding path?I think it would be smarter to track it some other way. Either keep
two lists of paths, one of which is the partial paths and the other of
which is the parallel-safe paths, or keep a bitmapset indicating which
paths fall into which category.I like two lists: it consumes almost no memory (two list headers
instead of one) compared to non-parallel-append when there are
non-partial paths and what more, it consumes no extra memory when all
paths are partial.
I agree that the two-lists approach will consume less memory than
bitmapset. Keeping two lists will effectively have an extra pointer
field which will add up to the AppendPath size, but this size will not
grow with the number of subpaths, whereas the Bitmapset will grow.
But as far as code is concerned, I think the two-list approach will
turn out to be less simple if we derive corresponding two different
arrays in AppendState node. Handling two different arrays during
execution does not look clean. Whereas, the bitmapset that I have used
in Append has turned out to be very simple. I just had to do the below
check (and that is the only location) to see if it's a partial or
non-partial subplan. There is nowhere else any special handling for
non-partial subpath.
/*
* Increment worker count for the chosen node, if at all we found one.
* For non-partial plans, set it to -1 instead, so that no other workers
* run it.
*/
if (min_whichplan != PA_INVALID_PLAN)
{
if (bms_is_member(min_whichplan,
((Append*)state->ps.plan)->partial_subplans_set))
padesc->pa_info[min_whichplan].pa_num_workers++;
else
padesc->pa_info[min_whichplan].pa_num_workers = -1;
}
Now, since Bitmapset field is used during execution with such
simplicity, why not have this same data structure in AppendPath, and
re-use bitmapset field in Append plan node without making a copy of
it. Otherwise, if we have two lists in AppendPath, and a bitmap in
Append, again there is going to be code for data structure conversion.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
But as far as code is concerned, I think the two-list approach will
turn out to be less simple if we derive corresponding two different
arrays in AppendState node. Handling two different arrays during
execution does not look clean. Whereas, the bitmapset that I have used
in Append has turned out to be very simple. I just had to do the below
check (and that is the only location) to see if it's a partial or
non-partial subplan. There is nowhere else any special handling for
non-partial subpath./*
* Increment worker count for the chosen node, if at all we found one.
* For non-partial plans, set it to -1 instead, so that no other workers
* run it.
*/
if (min_whichplan != PA_INVALID_PLAN)
{
if (bms_is_member(min_whichplan,
((Append*)state->ps.plan)->partial_subplans_set))
padesc->pa_info[min_whichplan].pa_num_workers++;
else
padesc->pa_info[min_whichplan].pa_num_workers = -1;
}Now, since Bitmapset field is used during execution with such
simplicity, why not have this same data structure in AppendPath, and
re-use bitmapset field in Append plan node without making a copy of
it. Otherwise, if we have two lists in AppendPath, and a bitmap in
Append, again there is going to be code for data structure conversion.
I think there is some merit in separating out non-parallel and
parallel plans within the same array or outside it. The current logic
to assign plan to a worker looks at all the plans, unnecessarily
hopping over the un-parallel ones after they are given to a worker. If
we separate those two, we can keep assigning new workers to the
non-parallel plans first and then iterate over the parallel ones when
a worker needs a plan to execute. We might eliminate the need for
special value -1 for num workers. You may separate those two kinds in
two different arrays or within the same array and remember the
smallest index of a parallel plan.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 10, 2017 at 11:33 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
But as far as code is concerned, I think the two-list approach will
turn out to be less simple if we derive corresponding two different
arrays in AppendState node. Handling two different arrays during
execution does not look clean. Whereas, the bitmapset that I have used
in Append has turned out to be very simple. I just had to do the below
check (and that is the only location) to see if it's a partial or
non-partial subplan. There is nowhere else any special handling for
non-partial subpath./*
* Increment worker count for the chosen node, if at all we found one.
* For non-partial plans, set it to -1 instead, so that no other workers
* run it.
*/
if (min_whichplan != PA_INVALID_PLAN)
{
if (bms_is_member(min_whichplan,
((Append*)state->ps.plan)->partial_subplans_set))
padesc->pa_info[min_whichplan].pa_num_workers++;
else
padesc->pa_info[min_whichplan].pa_num_workers = -1;
}Now, since Bitmapset field is used during execution with such
simplicity, why not have this same data structure in AppendPath, and
re-use bitmapset field in Append plan node without making a copy of
it. Otherwise, if we have two lists in AppendPath, and a bitmap in
Append, again there is going to be code for data structure conversion.I think there is some merit in separating out non-parallel and
parallel plans within the same array or outside it. The current logic
to assign plan to a worker looks at all the plans, unnecessarily
hopping over the un-parallel ones after they are given to a worker. If
we separate those two, we can keep assigning new workers to the
non-parallel plans first and then iterate over the parallel ones when
a worker needs a plan to execute. We might eliminate the need for
special value -1 for num workers. You may separate those two kinds in
two different arrays or within the same array and remember the
smallest index of a parallel plan.
Further to that, with this scheme and the scheme to distribute workers
equally irrespective of the maximum workers per plan, you don't need
to "scan" the subplans to find the one with minimum workers. If you
treat the array of parallel plans as a circular queue, the plan to be
assigned next to a worker will always be the plan next to the one
which got assigned to the given worker. Once you have assigned workers
to non-parallel plans, intialize a shared variable next_plan to point
to the first parallel plan. When a worker comes asking for a plan,
assign the plan pointed by next_plan and update it to the next plan in
the circular queue.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10 March 2017 at 12:33, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
On Fri, Mar 10, 2017 at 11:33 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:But as far as code is concerned, I think the two-list approach will
turn out to be less simple if we derive corresponding two different
arrays in AppendState node. Handling two different arrays during
execution does not look clean. Whereas, the bitmapset that I have used
in Append has turned out to be very simple. I just had to do the below
check (and that is the only location) to see if it's a partial or
non-partial subplan. There is nowhere else any special handling for
non-partial subpath./*
* Increment worker count for the chosen node, if at all we found one.
* For non-partial plans, set it to -1 instead, so that no other workers
* run it.
*/
if (min_whichplan != PA_INVALID_PLAN)
{
if (bms_is_member(min_whichplan,
((Append*)state->ps.plan)->partial_subplans_set))
padesc->pa_info[min_whichplan].pa_num_workers++;
else
padesc->pa_info[min_whichplan].pa_num_workers = -1;
}Now, since Bitmapset field is used during execution with such
simplicity, why not have this same data structure in AppendPath, and
re-use bitmapset field in Append plan node without making a copy of
it. Otherwise, if we have two lists in AppendPath, and a bitmap in
Append, again there is going to be code for data structure conversion.I think there is some merit in separating out non-parallel and
parallel plans within the same array or outside it. The current logic
to assign plan to a worker looks at all the plans, unnecessarily
hopping over the un-parallel ones after they are given to a worker. If
we separate those two, we can keep assigning new workers to the
non-parallel plans first and then iterate over the parallel ones when
a worker needs a plan to execute. We might eliminate the need for
special value -1 for num workers. You may separate those two kinds in
two different arrays or within the same array and remember the
smallest index of a parallel plan.
Do you think we might get performance benefit with this ? I am looking
more towards logic simplicity. non-parallel plans would be mostly
likely be there only in case of UNION ALL queries, and not partitioned
tables. And UNION ALL queries probably would have far lesser number of
subplans, there won't be too many unnecessary hops. The need for
num_workers=-1 will still be there for partial plans, because we need
to set it to -1 once a worker finishes a plan.
Further to that, with this scheme and the scheme to distribute workers
equally irrespective of the maximum workers per plan, you don't need
to "scan" the subplans to find the one with minimum workers. If you
treat the array of parallel plans as a circular queue, the plan to be
assigned next to a worker will always be the plan next to the one
which got assigned to the given worker.
Once you have assigned workers
to non-parallel plans, intialize a shared variable next_plan to point
to the first parallel plan. When a worker comes asking for a plan,
assign the plan pointed by next_plan and update it to the next plan in
the circular queue.
At some point of time, this logic may stop working. Imagine plans are
running with (1, 1, 1). Next worker goes to plan 1, so they run with
(2, 1, 1). So now the next_plan points to plan 2. Now suppose worker
on plan 2 finishes. It should not again take plan 2, even though
next_plan points to 2. It should take plan 3, or whichever is not
finished. May be a worker that finishes a plan should do this check
before directly going to the next_plan. But if this is turning out as
simple as the finding-min-worker-plan, we can use this logic. But will
have to check. We can anyway consider this even when we have a single
list.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
I think there is some merit in separating out non-parallel and
parallel plans within the same array or outside it. The current logic
to assign plan to a worker looks at all the plans, unnecessarily
hopping over the un-parallel ones after they are given to a worker. If
we separate those two, we can keep assigning new workers to the
non-parallel plans first and then iterate over the parallel ones when
a worker needs a plan to execute. We might eliminate the need for
special value -1 for num workers. You may separate those two kinds in
two different arrays or within the same array and remember the
smallest index of a parallel plan.Do you think we might get performance benefit with this ? I am looking
more towards logic simplicity. non-parallel plans would be mostly
likely be there only in case of UNION ALL queries, and not partitioned
tables. And UNION ALL queries probably would have far lesser number of
subplans, there won't be too many unnecessary hops.
A partitioned table which has foreign and local partitions would have
non-parallel and parallel plans if the foreign plans can not be
parallelized like what postgres_fdw does.
The need for
num_workers=-1 will still be there for partial plans, because we need
to set it to -1 once a worker finishes a plan.
IIRC, we do that so that no other workers are assigned to it when
scanning the array of plans. But with the new scheme we don't need to
scan the non-parallel plans for when assigning plan to workers so -1
may not be needed. I may be wrong though.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10 March 2017 at 14:05, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
The need for
num_workers=-1 will still be there for partial plans, because we need
to set it to -1 once a worker finishes a plan.IIRC, we do that so that no other workers are assigned to it when
scanning the array of plans. But with the new scheme we don't need to
scan the non-parallel plans for when assigning plan to workers so -1
may not be needed. I may be wrong though.
Still, when a worker finishes a partial subplan , it marks it as -1,
so that no new workers pick this, even if there are other workers
already executing it.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Moin,
On Fri, March 10, 2017 3:24 am, Amit Khandekar wrote:
On 10 March 2017 at 12:33, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:On Fri, Mar 10, 2017 at 11:33 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:But as far as code is concerned, I think the two-list approach will
turn out to be less simple if we derive corresponding two different
arrays in AppendState node. Handling two different arrays during
execution does not look clean. Whereas, the bitmapset that I have used
in Append has turned out to be very simple. I just had to do the below
check (and that is the only location) to see if it's a partial or
non-partial subplan. There is nowhere else any special handling for
non-partial subpath./*
* Increment worker count for the chosen node, if at all we found one.
* For non-partial plans, set it to -1 instead, so that no other
workers
* run it.
*/
if (min_whichplan != PA_INVALID_PLAN)
{
if (bms_is_member(min_whichplan,
((Append*)state->ps.plan)->partial_subplans_set))
padesc->pa_info[min_whichplan].pa_num_workers++;
else
padesc->pa_info[min_whichplan].pa_num_workers = -1;
}Now, since Bitmapset field is used during execution with such
simplicity, why not have this same data structure in AppendPath, and
re-use bitmapset field in Append plan node without making a copy of
it. Otherwise, if we have two lists in AppendPath, and a bitmap in
Append, again there is going to be code for data structure conversion.I think there is some merit in separating out non-parallel and
parallel plans within the same array or outside it. The current logic
to assign plan to a worker looks at all the plans, unnecessarily
hopping over the un-parallel ones after they are given to a worker. If
we separate those two, we can keep assigning new workers to the
non-parallel plans first and then iterate over the parallel ones when
a worker needs a plan to execute. We might eliminate the need for
special value -1 for num workers. You may separate those two kinds in
two different arrays or within the same array and remember the
smallest index of a parallel plan.Do you think we might get performance benefit with this ? I am looking
more towards logic simplicity. non-parallel plans would be mostly
likely be there only in case of UNION ALL queries, and not partitioned
tables. And UNION ALL queries probably would have far lesser number of
subplans, there won't be too many unnecessary hops. The need for
num_workers=-1 will still be there for partial plans, because we need
to set it to -1 once a worker finishes a plan.Further to that, with this scheme and the scheme to distribute workers
equally irrespective of the maximum workers per plan, you don't need
to "scan" the subplans to find the one with minimum workers. If you
treat the array of parallel plans as a circular queue, the plan to be
assigned next to a worker will always be the plan next to the one
which got assigned to the given worker.Once you have assigned workers
to non-parallel plans, intialize a shared variable next_plan to point
to the first parallel plan. When a worker comes asking for a plan,
assign the plan pointed by next_plan and update it to the next plan in
the circular queue.At some point of time, this logic may stop working. Imagine plans are
running with (1, 1, 1). Next worker goes to plan 1, so they run with
(2, 1, 1). So now the next_plan points to plan 2. Now suppose worker
on plan 2 finishes. It should not again take plan 2, even though
next_plan points to 2. It should take plan 3, or whichever is not
finished. May be a worker that finishes a plan should do this check
before directly going to the next_plan. But if this is turning out as
simple as the finding-min-worker-plan, we can use this logic. But will
have to check. We can anyway consider this even when we have a single
list.
Just a question for me to understand the implementation details vs. the
strategy:
Have you considered how the scheduling decision might impact performance
due to "inter-plan parallelism vs. in-plan parallelism"?
So what would be the scheduling strategy? And should there be a fixed one
or user-influencable? And what could be good ones?
A simple example:
E.g. if we have 5 subplans, and each can have at most 5 workers and we
have 5 workers overall.
So, do we:
Assign 5 workers to plan 1. Let it finish.
Then assign 5 workers to plan 2. Let it finish.
and so on
or:
Assign 1 workers to each plan until no workers are left?
In the second case you would have 5 plans running in a quasy-sequential
manner, which might be slower than the other way. Or not, that probably
needs some benchmarks?
Likewise, if you have a mix of plans with max workers like:
Plan A: 1 worker
Plan B: 2 workers
Plan C: 3 workers
Plan D: 1 worker
Plan E: 4 workers
Would the strategy be:
* Serve them in first-come-first-served order? (A,B,C,D?) (Would order
here be random due to how the plan's emerge, i.e. could the user re-order
query to get a different order?)
* Serve them in max-workers order? (A,D,B,C)
* Serve first all with 1 worker, then fill the rest? (A,D,B,C | A,D,C,B)
* Serve them by some other metric, e.g. index-only scans first, seq-scans
last? Or a mix of all these?
Excuse me if I just didn't see this from the thread so far. :)
Best regards,
Tels
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
After giving more thought to our discussions, I have have used the
Bitmapset structure in AppendPath as against having two lists one for
partial and other for non-partial paths. Attached is the patch v6 that
has the required changes. So accumulate_append_subpath() now also
prepares the bitmapset containing the information about which paths
are partial paths. This is what I had done in the first version.
At this point of time, I have not given sufficient time to think about
Ashutosh's proposal of just keeping track of the next_subplan which he
mentioned. There, we just keep assigning workers to a circle of
subplans in round-robin style. But I think as of now the approach of
choosing the minimum worker subplan is pretty simple looking. So the
patch v6 is in a working condition using minimum-worker approach.
On 9 March 2017 at 07:22, Robert Haas <robertmhaas@gmail.com> wrote:
Some review:
+typedef struct ParallelAppendDescData +{ + slock_t pa_mutex; /* mutual exclusion to choose next subplan */ + ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER]; +} ParallelAppendDescData;Instead of having ParallelAppendInfo, how about just int
pa_workers[FLEXIBLE_ARRAY_MEMBER]? The second structure seems like
overkill, at least for now.
I have , for now, kept the structure there, just in case after further
discussion we may add something.
+static inline void +exec_append_scan_first(AppendState *appendstate) +{ + appendstate->as_whichplan = 0; +}I don't think this is buying you anything, and suggest backing it out.
This is required for sequential Append, so that we can start executing
from the first subplan.
+ /* Backward scan is not supported by parallel-aware plans */ + Assert(!ScanDirectionIsBackward(appendstate->ps.state->es_direction));I think you could assert ScanDirectionIsForward, couldn't you?
NoMovement, I assume, is right out.
Right. Changed.
+ elog(DEBUG2, "ParallelAppend : pid %d : all plans already finished", + MyProcPid);Please remove (and all similar cases also).
Removed at multiple places.
+ sizeof(*node->as_padesc->pa_info) * node->as_nplans);
I'd use the type name instead.
Done.
+ for (i = 0; i < node->as_nplans; i++) + { + /* + * Just setting all the number of workers to 0 is enough. The logic + * of choosing the next plan in workers will take care of everything + * else. + */ + padesc->pa_info[i].pa_num_workers = 0; + }Here I'd use memset.
Done.
+ return (min_whichplan == PA_INVALID_PLAN ? false : true);
Maybe just return (min_whichplan != PA_INVALID_PLAN);
Done.
- childrel->cheapest_total_path); + childrel->cheapest_total_path);Unnecessary.
This call is now having more param, so kept the change.
+ {
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
+ }Don't need to add braces.
Removed them.
+ /* + * Extract the first unparameterized, parallel-safe one among the + * child paths. + */Can we use get_cheapest_parallel_safe_total_inner for this, from
a71f10189dc10a2fe422158a2c9409e0f77c6b9e?
Yes, Fixed.
+ if (rel->partial_pathlist != NIL && + (Path *) linitial(rel->partial_pathlist) == subpath) + partial_subplans_set = bms_add_member(partial_subplans_set, i);This seems like a scary way to figure this out. What if we wanted to
build a parallel append subpath with some path other than the
cheapest, for some reason? I think you ought to record the decision
that set_append_rel_pathlist makes about whether to use a partial path
or a parallel-safe path, and then just copy it over here.
As mentioned above, used Bitmapset in AppendPath.
- create_append_path(grouped_rel, - paths, - NULL, - 0); + create_append_path(grouped_rel, paths, NULL, 0);Unnecessary.
Now since there was anyway a change in the number of params, I kept
the single line call.
Please refer to attached patch version v6 for all of the above changes.
Attachments:
ParallelAppend_v6.patchapplication/octet-stream; name=ParallelAppend_v6.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index a1289e5..41d807c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -214,6 +215,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -278,6 +283,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -781,6 +790,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..f156907 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,56 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendInfo
+{
+ /*
+ * pa_num_workers : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_num_workers to -1, so that no new
+ * worker pick this subplan. For non-partial subplan, a worker which picks
+ * up that subplan, it should immediately set to -1, so as to make sure
+ * there are no more than 1 worker assigned to this subplan. In general, -1
+ * means workers should stop picking it.
+ */
+ int pa_num_workers;
+
+} ParallelAppendInfo;
+
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ ParallelAppendInfo pa_info[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+
+
+static void exec_append_scan_first(AppendState *appendstate);
static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static inline void
+exec_append_scan_first(AppendState *appendstate)
+{
+ appendstate->as_whichplan = 0;
+}
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -77,6 +124,27 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ {
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(appendstate->ps.state->es_direction));
+
+ return parallel_append_next(appendstate);
+ }
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -178,8 +246,7 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
/*
* initialize to scan first subplan
*/
- appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
+ exec_append_scan_first(appendstate);
return appendstate;
}
@@ -198,6 +265,10 @@ ExecAppend(AppendState *node)
PlanState *subnode;
TupleTableSlot *result;
+ /* Check if we are already finished plans from parallel append */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
/*
* figure out which subplan we are currently processing
*/
@@ -219,14 +290,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -270,6 +345,7 @@ ExecReScanAppend(AppendState *node)
for (i = 0; i < node->as_nplans; i++)
{
PlanState *subnode = node->appendplans[i];
+ ParallelAppendDesc padesc = node->as_padesc;
/*
* ExecReScan doesn't know about my subplans, so I have to do
@@ -284,7 +360,195 @@ ExecReScanAppend(AppendState *node)
*/
if (subnode->chgParam == NULL)
ExecReScan(subnode);
+
+ if (padesc)
+ {
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan will take care of everything else.
+ * pa_max_workers is already set initially.
+ */
+ padesc->pa_info[i].pa_num_workers = 0;
+ }
}
- node->as_whichplan = 0;
- exec_append_initialize_next(node);
+
+ exec_append_scan_first(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_info),
+ sizeof(ParallelAppendInfo) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ memset(padesc->pa_info, 0, sizeof(ParallelAppendInfo) * node->as_nplans);
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_info[whichplan].pa_num_workers = -1;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the optimal subplan that should be executed. The logic is to
+ * choose the subplan that is being executed by the least number of
+ * workers.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int min_whichplan = PA_INVALID_PLAN;
+ int min_workers = -1; /* Keep compiler quiet */
+
+ Assert(padesc != NULL);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Choose the plan with the least number of workers */
+ for (whichplan = 0; whichplan < state->as_nplans; whichplan++)
+ {
+ ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
+
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (painfo->pa_num_workers == -1)
+ continue;
+
+ /*
+ * Keep track of the node with the least workers so far. For the very
+ * first plan, choose that one as the least-workers node.
+ */
+ if (min_whichplan == PA_INVALID_PLAN ||
+ painfo->pa_num_workers < min_workers)
+ {
+ min_whichplan = whichplan;
+ min_workers = painfo->pa_num_workers;
+ }
+ }
+
+
+ /*
+ * Increment worker count for the chosen node, if at all we found one.
+ * For non-partial plans, set it to -1 instead, so that no other workers
+ * run it.
+ */
+ if (min_whichplan != PA_INVALID_PLAN)
+ {
+ if (bms_is_member(min_whichplan,
+ ((Append*)state->ps.plan)->partial_subplans_set))
+ padesc->pa_info[min_whichplan].pa_num_workers++;
+ else
+ padesc->pa_info[min_whichplan].pa_num_workers = -1;
+ }
+
+ /*
+ * Save the chosen plan index. It can be PA_INVALID_PLAN, which means we
+ * are done with all nodes (Note : this meaning applies only to *parallel*
+ * append).
+ */
+ state->as_whichplan = min_whichplan;
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+ SpinLockRelease(&padesc->pa_mutex);
+
+ /*
+ * If we didn't find any node to work on, stop executing. Indicate the same
+ * by returning false.
+ */
+ return (min_whichplan != PA_INVALID_PLAN);
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index bfc2ac1..f861881 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_BITMAPSET_FIELD(partial_subplans_set);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 7418fbe..0aea119 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_BITMAPSET_FIELD(partial_subplans_set);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d3bbc02..bb366bc 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1565,6 +1565,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_BITMAPSET_FIELD(partial_subplans_set);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b263359..16c2f5b 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -99,7 +99,8 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
-static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_append_subpath(List *subpaths, Path *path,
+ Bitmapset **partial_subpaths_set);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1186,6 +1187,7 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
bool subpaths_valid = true;
List *partial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ Bitmapset *partial_subpath_set = NULL;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1245,14 +1247,41 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path,
+ NULL);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
- linitial(childrel->partial_pathlist));
+ linitial(childrel->partial_pathlist),
+ &partial_subpath_set);
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ Path *parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_append_subpath(partial_subpaths,
+ parallel_safe_path, NULL);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1327,7 +1356,8 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, subpaths,
+ NULL, NULL, 0));
/*
* Consider an append of partial unordered, unparameterized partial paths.
@@ -1335,26 +1365,13 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
+ int parallel_workers;
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
-
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers);
+ appendpath = create_append_path(rel, partial_subpaths,
+ partial_subpath_set,
+ NULL, parallel_workers);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1401,12 +1418,13 @@ set_append_rel_pathlist(PlannerInfo *root, RelOptInfo *rel,
subpaths_valid = false;
break;
}
- subpaths = accumulate_append_subpath(subpaths, subpath);
+ subpaths = accumulate_append_subpath(subpaths, subpath, NULL);
}
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0));
+ create_append_path(rel, subpaths,
+ NULL, required_outer, 0));
}
}
@@ -1490,9 +1508,11 @@ generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
startup_neq_total = true;
startup_subpaths =
- accumulate_append_subpath(startup_subpaths, cheapest_startup);
+ accumulate_append_subpath(startup_subpaths,
+ cheapest_startup, NULL);
total_subpaths =
- accumulate_append_subpath(total_subpaths, cheapest_total);
+ accumulate_append_subpath(total_subpaths,
+ cheapest_total, NULL);
}
/* ... and build the MergeAppend paths */
@@ -1584,6 +1604,43 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
return cheapest;
}
+/* concat_append_subpaths
+ * helper function for accumulate_append_subpath()
+ *
+ * child_partial_subpaths_set is the bitmap set to indicate which of the
+ * childpaths are partial paths. This is currently non-NULL only in case
+ * the childpaths belong to an Append path.
+ */
+static List *
+concat_append_subpaths(List *append_subpaths, List *childpaths,
+ Bitmapset **partial_subpaths_set,
+ Bitmapset *child_partial_subpaths_set)
+{
+ int i;
+ int append_subpath_len = list_length(append_subpaths);
+
+ if (partial_subpaths_set)
+ {
+ for (i = 0; i < list_length(childpaths); i++)
+ {
+ /*
+ * The child paths themselves may or may not be partial paths. The
+ * bitmapset numbers of these paths will need to be set considering
+ * that these are to be appended onto the partial_subpaths_set.
+ */
+ if (!child_partial_subpaths_set ||
+ bms_is_member(i, child_partial_subpaths_set))
+ {
+ *partial_subpaths_set = bms_add_member(*partial_subpaths_set,
+ append_subpath_len + i);
+ }
+ }
+ }
+
+ /* list_copy is important here to avoid sharing list substructure */
+ return list_concat(append_subpaths, list_copy(childpaths));
+}
+
/*
* accumulate_append_subpath
* Add a subpath to the list being built for an Append or MergeAppend
@@ -1597,26 +1654,34 @@ get_cheapest_parameterized_child_path(PlannerInfo *root, RelOptInfo *rel,
* omitting a sort step, which seems fine: if the parent is to be an Append,
* its result would be unsorted anyway, while if the parent is to be a
* MergeAppend, there's no point in a separate sort on a child.
+ *
+ * If partial_subpaths_set is not NULL, it means we are building a
+ * partial subpaths list, and so we need to add the path (or its child paths
+ * in case it's Append or MergeAppend) into the partial_subpaths bitmap set.
*/
static List *
-accumulate_append_subpath(List *subpaths, Path *path)
+accumulate_append_subpath(List *append_subpaths, Path *path,
+ Bitmapset **partial_subpaths_set)
{
if (IsA(path, AppendPath))
{
- AppendPath *apath = (AppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(apath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((AppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ ((AppendPath*)path)->partial_subpaths);
}
else if (IsA(path, MergeAppendPath))
{
- MergeAppendPath *mpath = (MergeAppendPath *) path;
-
- /* list_copy is important here to avoid sharing list substructure */
- return list_concat(subpaths, list_copy(mpath->subpaths));
+ return concat_append_subpaths(append_subpaths,
+ ((MergeAppendPath*)path)->subpaths,
+ partial_subpaths_set,
+ NULL);
}
else
- return lappend(subpaths, path);
+ return concat_append_subpaths(append_subpaths,
+ list_make1(path),
+ partial_subpaths_set,
+ NULL);
}
/*
@@ -1639,7 +1704,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index e78f3a8..c5da25c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1697,6 +1698,70 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up.
+ */
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0d00683..daf45c3 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1197,7 +1197,7 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NULL, NULL, 0));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index d002e6d..ce4c1dd 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, Bitmapset *partial_plans_set,
+ List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, best_path->partial_subpaths, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5187,7 +5188,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, Bitmapset *partial_plans_set, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5197,6 +5198,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->partial_subplans_set = partial_plans_set;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 02286d9..2b747c8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3345,10 +3345,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NULL, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 1389db1..07ea748 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NULL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 8ce772d..574dc9e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1193,6 +1193,67 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) i.e. ln(num_subpaths) / ln(2) */
+ log2w = log(list_length(subpaths)) / 0.693 ;
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1261,9 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,40 +1273,28 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->subpaths = subpaths;
+ pathnode->partial_subpaths = partial_subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, subpaths);
+
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 811ea51..8aecbff 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index b880dc1..41de095 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ Bitmapset *partial_subplans_set;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 05d6f07..be3abda 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1117,6 +1117,7 @@ typedef struct AppendPath
{
Path path;
List *subpaths; /* list of component Paths */
+ Bitmapset *partial_subpaths;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..1f42850 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,7 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 373c722..ee1ecb4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,8 +64,10 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, Bitmapset *partial_subpaths,
+ Relids required_outer, int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 795d9f5..367d23f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1337,6 +1337,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1403,6 +1404,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..ba963e6 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -17,9 +17,9 @@ explain (costs off)
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index 836ec22..0636f08 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -462,11 +462,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
On Fri, Mar 10, 2017 at 12:17 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
I agree that the two-lists approach will consume less memory than
bitmapset. Keeping two lists will effectively have an extra pointer
field which will add up to the AppendPath size, but this size will not
grow with the number of subpaths, whereas the Bitmapset will grow.
Sure. You'll use about one BIT of memory per subpath. I'm kind of
baffled as to why we're treating this as an issue worth serious
discussion; the amount of memory involved is clearly very small. Even
for an appendrel with 1000 children, that's 125 bytes of memory.
Considering the amount of memory we're going to spend planning that
appendrel overall, that's not significant.
However, Ashutosh's response made me think of something: one thing is
that we probably do want to group all of the non-partial plans at the
beginning of the Append so that they get workers first, and put the
partial plans afterward. That's because the partial plans can always
be accelerated by adding more workers as they become available, but
the non-partial plans are just going to take as long as they take - so
we want to start them as soon as possible. In fact, what we might
want to do is actually sort the non-partial paths in order of
decreasing cost, putting the most expensive one first and the others
in decreasing order after that - and then similarly afterward with the
partial paths. If we did that, we wouldn't need to store a bitmapset
OR two separate lists. We could just store the index of the first
partial plan in the list. Then you can test whether a path is partial
by checking whether this_index >= first_partial_index.
One problem with that is that, since the leader has about a 4ms head
start on the other workers, it would tend to pick the most expensive
path to run locally before any other worker had a chance to make a
selection, and that's probably not what we want. To fix that, let's
have the leader start at the end of the list of plans and work
backwards towards the beginning, so that it prefers cheaper and
partial plans over decisions that would force it to undertake a large
amount of work itself.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 10, 2017 at 8:12 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
+static inline void +exec_append_scan_first(AppendState *appendstate) +{ + appendstate->as_whichplan = 0; +}I don't think this is buying you anything, and suggest backing it out.
This is required for sequential Append, so that we can start executing
from the first subplan.
My point is that there's really no point in defining a static inline
function containing one line of code. You could just put that line of
code in whatever places need it, which would probably be more clear.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 10, 2017 at 6:01 AM, Tels <nospam-pg-abuse@bloodgate.com> wrote:
Just a question for me to understand the implementation details vs. the
strategy:Have you considered how the scheduling decision might impact performance
due to "inter-plan parallelism vs. in-plan parallelism"?So what would be the scheduling strategy? And should there be a fixed one
or user-influencable? And what could be good ones?A simple example:
E.g. if we have 5 subplans, and each can have at most 5 workers and we
have 5 workers overall.So, do we:
Assign 5 workers to plan 1. Let it finish.
Then assign 5 workers to plan 2. Let it finish.
and so onor:
Assign 1 workers to each plan until no workers are left?
Currently, we do the first of those, but I'm pretty sure the second is
way better. For example, suppose each subplan has a startup cost. If
you have all the workers pile on each plan in turn, every worker pays
the startup cost for every subplan. If you spread them out, then
subplans can get finished without being visited by all workers, and
then the other workers never pay those costs. Moreover, you reduce
contention for spinlocks, condition variables, etc. It's not
impossible to imagine a scenario where having all workers pile on one
subplan at a time works out better: for example, suppose you have a
table with lots of partitions all of which are on the same disk, and
it's actually one physical spinning disk, not an SSD or a disk array
or anything, and the query is completely I/O-bound. Well, it could
be, in that scenario, that spreading out the workers is going to turn
sequential I/O into random I/O and that might be terrible. In most
cases, though, I think you're going to be better off. If the
partitions are on different spindles or if there's some slack I/O
capacity for prefetching, you're going to come out ahead, maybe way
ahead. If you come out behind, then you're evidently totally I/O
bound and have no capacity for I/O parallelism; in that scenario, you
should probably just turn parallel query off altogether, because
you're not going to benefit from it.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Moin,
On Sat, March 11, 2017 11:29 pm, Robert Haas wrote:
On Fri, Mar 10, 2017 at 6:01 AM, Tels <nospam-pg-abuse@bloodgate.com>
wrote:Just a question for me to understand the implementation details vs. the
strategy:Have you considered how the scheduling decision might impact performance
due to "inter-plan parallelism vs. in-plan parallelism"?So what would be the scheduling strategy? And should there be a fixed
one
or user-influencable? And what could be good ones?A simple example:
E.g. if we have 5 subplans, and each can have at most 5 workers and we
have 5 workers overall.So, do we:
Assign 5 workers to plan 1. Let it finish.
Then assign 5 workers to plan 2. Let it finish.
and so onor:
Assign 1 workers to each plan until no workers are left?
Currently, we do the first of those, but I'm pretty sure the second is
way better. For example, suppose each subplan has a startup cost. If
you have all the workers pile on each plan in turn, every worker pays
the startup cost for every subplan. If you spread them out, then
subplans can get finished without being visited by all workers, and
then the other workers never pay those costs. Moreover, you reduce
contention for spinlocks, condition variables, etc. It's not
impossible to imagine a scenario where having all workers pile on one
subplan at a time works out better: for example, suppose you have a
table with lots of partitions all of which are on the same disk, and
it's actually one physical spinning disk, not an SSD or a disk array
or anything, and the query is completely I/O-bound. Well, it could
be, in that scenario, that spreading out the workers is going to turn
sequential I/O into random I/O and that might be terrible. In most
cases, though, I think you're going to be better off. If the
partitions are on different spindles or if there's some slack I/O
capacity for prefetching, you're going to come out ahead, maybe way
ahead. If you come out behind, then you're evidently totally I/O
bound and have no capacity for I/O parallelism; in that scenario, you
should probably just turn parallel query off altogether, because
you're not going to benefit from it.
I agree with the proposition that both strategies can work well, or not,
depending on system-setup, the tables and data layout. I'd be a bit more
worried about turning it into the "random-io-case", but that's still just
a feeling and guesswork.
So which one will be better seems speculative, hence the question for
benchmarking different strategies.
So, I'd like to see the scheduler be out in a single place, maybe a
function that get's called with the number of currently running workers,
the max. number of workers to be expected, the new worker, the list of
plans still todo, and then schedules that single worker to one of these
plans by strategy X.
That would make it easier to swap out X for Y and see how it fares,
wouldn't it?
However, I don't think the patch needs to select the optimal strategy
right from the beginning (if that even exists, maybe it's a mixed
strategy), even "not so optimal" parallelism will be better than doing all
things sequentially.
Best regards,
Tels
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10 March 2017 at 22:08, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Mar 10, 2017 at 12:17 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
I agree that the two-lists approach will consume less memory than
bitmapset. Keeping two lists will effectively have an extra pointer
field which will add up to the AppendPath size, but this size will not
grow with the number of subpaths, whereas the Bitmapset will grow.Sure. You'll use about one BIT of memory per subpath. I'm kind of
baffled as to why we're treating this as an issue worth serious
discussion; the amount of memory involved is clearly very small. Even
for an appendrel with 1000 children, that's 125 bytes of memory.
Considering the amount of memory we're going to spend planning that
appendrel overall, that's not significant.
Yes, I agree that we should consider rather other things like code
simplicity to determine which data structure we should use in
AppendPath.
However, Ashutosh's response made me think of something: one thing is
that we probably do want to group all of the non-partial plans at the
beginning of the Append so that they get workers first, and put the
partial plans afterward. That's because the partial plans can always
be accelerated by adding more workers as they become available, but
the non-partial plans are just going to take as long as they take - so
we want to start them as soon as possible. In fact, what we might
want to do is actually sort the non-partial paths in order of
decreasing cost, putting the most expensive one first and the others
in decreasing order after that - and then similarly afterward with the
partial paths. If we did that, we wouldn't need to store a bitmapset
OR two separate lists. We could just store the index of the first
partial plan in the list. Then you can test whether a path is partial
by checking whether this_index >= first_partial_index.
I agree that we should preferably have the non-partial plans started
first. But I am not sure if it is really worth ordering the partial
plans by cost. The reason we ended up not keeping track of the
per-subplan parallel_worker, is because it would not matter much ,
and we would just equally distribute the workers among all regardless
of how big the subplans are. Even if smaller plans get more worker,
they will finish faster, and workers would be available to larger
subplans sooner.
Anyways, I have given a thought on the logic of choosing the next plan
, and that is irrespective of whether the list is sorted. I have
included Ashutosh's proposal of scanning the array round-robin as
against finding the minimum, since that method will automatically
distribute the workers evenly. Also, the logic uses a single array and
keeps track of first partial plan. The first section of the array is
non-partial, followed by partial plans. Below is the algorithm ...
There might be corner cases which I didn't yet take into account, but
first I wanted to get an agreement if this looks ok to go ahead with.
Since it does not find minimum worker count, it no longer uses
pa_num_workers. Instead it has boolean field painfo->pa_finished.
parallel_append_next(AppendState *state)
{
/* Make a note of which subplan we have started with */
initial_plan = padesc->next_plan;
/* Keep going to the next plan until we find an unfinished one. In
the process, also keep track of the first unfinished subplan. As the
non-partial subplans are taken one by one, the unfinished subplan will
shift ahead, so that we don't have to scan these anymore */
whichplan = initial_plan;
for (;;)
{
ParallelAppendInfo *painfo = &padesc->pa_info[whichplan];
/*
* Ignore plans that are already done processing. These also include
* non-partial subplans which have already been taken by a worker.
*/
if (!painfo->pa_finished)
{
/* If this a non-partial plan, immediately mark it
finished, and shift ahead first_plan */
if (whichplan < padesc->first_partial_plan)
{
padesc->pa_info[whichplan].pa_finished = true;
padesc->first_plan++;
}
break;
}
/* Either go to the next index, or wrap around to the first
unfinished one */
whichplan = goto_next_plan(whichplan, padesc->first_plan,
padesc->as_nplans - 1));
/* Have we scanned all subplans ? If yes, we are done */
if (whichplan == initial_plan)
break;
}
/* If we didn't find any plan to execute, stop executing. */
if (whichplan == initial_plan || whichplan == PA_INVALID_PLAN)
return false;
else
{
/* Set the chosen plan, and also the next plan to be picked by
other workers */
state->as_whichplan = whichplan;
padesc->next_plan = goto_next_plan(whichplan,
padesc->first_plan, padesc->as_nplans - 1));
return true;
}
}
/* Either go to the next index, or wrap around to the first unfinished one */
int goto_next_plan(curplan, first_plan, last_plan)
{
if (curplan + 1 <= last_plan)
return curplan + 1;
else
return first_plan;
}
One problem with that is that, since the leader has about a 4ms head
start on the other workers, it would tend to pick the most expensive
path to run locally before any other worker had a chance to make a
selection, and that's probably not what we want. To fix that, let's
have the leader start at the end of the list of plans and work
backwards towards the beginning, so that it prefers cheaper and
partial plans over decisions that would force it to undertake a large
amount of work itself.--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 12 March 2017 at 19:31, Tels <nospam-pg-abuse@bloodgate.com> wrote:
Moin,
On Sat, March 11, 2017 11:29 pm, Robert Haas wrote:
On Fri, Mar 10, 2017 at 6:01 AM, Tels <nospam-pg-abuse@bloodgate.com>
wrote:Just a question for me to understand the implementation details vs. the
strategy:Have you considered how the scheduling decision might impact performance
due to "inter-plan parallelism vs. in-plan parallelism"?So what would be the scheduling strategy? And should there be a fixed
one
or user-influencable? And what could be good ones?A simple example:
E.g. if we have 5 subplans, and each can have at most 5 workers and we
have 5 workers overall.So, do we:
Assign 5 workers to plan 1. Let it finish.
Then assign 5 workers to plan 2. Let it finish.
and so onor:
Assign 1 workers to each plan until no workers are left?
Currently, we do the first of those, but I'm pretty sure the second is
way better. For example, suppose each subplan has a startup cost. If
you have all the workers pile on each plan in turn, every worker pays
the startup cost for every subplan. If you spread them out, then
subplans can get finished without being visited by all workers, and
then the other workers never pay those costs. Moreover, you reduce
contention for spinlocks, condition variables, etc. It's not
impossible to imagine a scenario where having all workers pile on one
subplan at a time works out better: for example, suppose you have a
table with lots of partitions all of which are on the same disk, and
it's actually one physical spinning disk, not an SSD or a disk array
or anything, and the query is completely I/O-bound. Well, it could
be, in that scenario, that spreading out the workers is going to turn
sequential I/O into random I/O and that might be terrible. In most
cases, though, I think you're going to be better off. If the
partitions are on different spindles or if there's some slack I/O
capacity for prefetching, you're going to come out ahead, maybe way
ahead. If you come out behind, then you're evidently totally I/O
bound and have no capacity for I/O parallelism; in that scenario, you
should probably just turn parallel query off altogether, because
you're not going to benefit from it.I agree with the proposition that both strategies can work well, or not,
depending on system-setup, the tables and data layout. I'd be a bit more
worried about turning it into the "random-io-case", but that's still just
a feeling and guesswork.So which one will be better seems speculative, hence the question for
benchmarking different strategies.So, I'd like to see the scheduler be out in a single place, maybe a
function that get's called with the number of currently running workers,
the max. number of workers to be expected, the new worker, the list of
plans still todo, and then schedules that single worker to one of these
plans by strategy X.That would make it easier to swap out X for Y and see how it fares,
wouldn't it?
Yes, actually pretty much the scheduler logic is all in one single
function parallel_append_next().
However, I don't think the patch needs to select the optimal strategy
right from the beginning (if that even exists, maybe it's a mixed
strategy), even "not so optimal" parallelism will be better than doing all
things sequentially.Best regards,
Tels
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 13, 2017 at 4:59 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
I agree that we should preferably have the non-partial plans started
first. But I am not sure if it is really worth ordering the partial
plans by cost. The reason we ended up not keeping track of the
per-subplan parallel_worker, is because it would not matter much ,
and we would just equally distribute the workers among all regardless
of how big the subplans are. Even if smaller plans get more worker,
they will finish faster, and workers would be available to larger
subplans sooner.
Imagine that the plan costs are 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, and 10
and you have 2 workers.
If you move that 10 to the front, this will finish in 10 time units.
If you leave it at the end, it will take 15 time units.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 13, 2017 at 7:46 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, Mar 13, 2017 at 4:59 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
I agree that we should preferably have the non-partial plans started
first. But I am not sure if it is really worth ordering the partial
plans by cost. The reason we ended up not keeping track of the
per-subplan parallel_worker, is because it would not matter much ,
and we would just equally distribute the workers among all regardless
of how big the subplans are. Even if smaller plans get more worker,
they will finish faster, and workers would be available to larger
subplans sooner.Imagine that the plan costs are 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, and 10
and you have 2 workers.If you move that 10 to the front, this will finish in 10 time units.
If you leave it at the end, it will take 15 time units.
Oh, never mind. You were only asking whether we should sort partial
plans. That's a lot less important, and maybe not important at all.
The only consideration there is whether we might try to avoid having
the leader start in on a plan with a large startup cost.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 12 March 2017 at 08:50, Robert Haas <robertmhaas@gmail.com> wrote:
However, Ashutosh's response made me think of something: one thing is
that we probably do want to group all of the non-partial plans at the
beginning of the Append so that they get workers first, and put the
partial plans afterward. That's because the partial plans can always
be accelerated by adding more workers as they become available, but
the non-partial plans are just going to take as long as they take - so
we want to start them as soon as possible. In fact, what we might
want to do is actually sort the non-partial paths in order of
decreasing cost, putting the most expensive one first and the others
in decreasing order after that - and then similarly afterward with the
partial paths. If we did that, we wouldn't need to store a bitmapset
OR two separate lists. We could just store the index of the first
partial plan in the list. Then you can test whether a path is partial
by checking whether this_index >= first_partial_index.
Attached is an updated patch v7, which does the above. Now,
AppendState->subplans has all non-partial subplans followed by all
partial subplans, with the non-partial subplans in the order of
descending total cost. Also, for convenience, the AppendPath also now
has similar ordering in its AppendPath->subpaths. So there is a new
field both in Append and AppendPath : first_partial_path/plan, which
has value 0 if there are no non-partial subpaths.
Also the backend now scans reverse, so that it does not take up the
most expensive path.
There are also some changes in the costing done. Now that we know that
the very first path is the costliest non-partial path, we can use its
total cost as the total cost of Append in case all the partial path
costs are lesser.
Modified/enhanced an existing test scenario in
src/test/regress/select_parallel.sql so that Parallel Append is
covered.
As suggested by Robert, since pa_info->pa_finished was the only field
in pa_info, removed the ParallelAppendDescData.pa_info structure, and
instead brought pa_info->pa_finished into ParallelAppendDescData.
+static inline void +exec_append_scan_first(AppendState *appendstate) +{ + appendstate->as_whichplan = 0; +}I don't think this is buying you anything, and suggest backing it out.
This is required for sequential Append, so that we can start executing
from the first subplan.My point is that there's really no point in defining a static inline
function containing one line of code. You could just put that line of
code in whatever places need it, which would probably be more clear.
Did the same.
Attachments:
ParallelAppend_v7.patchapplication/octet-stream; name=ParallelAppend_v7.patchDownload
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index a1289e5..41d807c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -214,6 +215,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -278,6 +283,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -781,6 +790,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index 6986cae..254e5ed 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,10 +59,48 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
-static bool exec_append_initialize_next(AppendState *appendstate);
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ slock_t pa_mutex; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+static bool exec_append_initialize_next(AppendState *appendstate);
+static void set_finished(ParallelAppendDesc padesc, int whichplan);
+static bool parallel_append_next(AppendState *state);
+static bool leader_next(AppendState *state);
+static int goto_next_plan(int curplan, int first_plan, int last_plan);
+
/* ----------------------------------------------------------------
* exec_append_initialize_next
*
@@ -77,6 +115,27 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ {
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(appendstate->ps.state->es_direction));
+
+ return parallel_append_next(appendstate);
+ }
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -176,10 +235,9 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * In case it's a sequential Append, initialize to scan first subplan.
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +257,14 @@ ExecAppend(AppendState *node)
TupleTableSlot *result;
/*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+ /*
* figure out which subplan we are currently processing
*/
subnode = node->appendplans[node->as_whichplan];
@@ -219,14 +285,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ set_finished(node->as_padesc, node->as_whichplan);
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -266,6 +336,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -285,6 +356,284 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+ SpinLockInit(&padesc->pa_mutex);
+
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * set_finished
+ *
+ * Indicate that this child plan node is about to be finished, so no other
+ * workers should take up this node. Workers who are already executing
+ * this node will continue to do so, but workers looking for next nodes to
+ * pick up would skip this node after this function is called. It is
+ * possible that multiple workers call this function for the same node at
+ * the same time, because these workers were executing the same node and
+ * they finished with it at the same time. The spinlock is not for this
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
+ * ----------------------------------------------------------------
+ */
+static void
+set_finished(ParallelAppendDesc padesc, int whichplan)
+{
+ SpinLockAcquire(&padesc->pa_mutex);
+ padesc->pa_finished[whichplan] = true;
+ SpinLockRelease(&padesc->pa_mutex);
+}
+
+/* ----------------------------------------------------------------
+ * parallel_append_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+parallel_append_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return leader_next(state);
+
+ SpinLockAcquire(&padesc->pa_mutex);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = goto_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ /* If we didn't find any plan to execute, stop executing. */
+ if (!found)
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = goto_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ SpinLockRelease(&padesc->pa_mutex);
+
+ /*
+ * Note: There is a chance that just after the child plan node is chosen
+ * here and spinlock released, some other worker finishes this node and
+ * calls set_finished(). In that case, this worker will go ahead and call
+ * ExecProcNode(child_node), which will return NULL tuple since it is
+ * already finished, and then once again this worker will try to choose
+ * next subplan; but this is ok : it's just an extra "choose_next_subplan"
+ * operation.
+ */
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ /* The parallel leader should start from the last subplan. */
+
+ SpinLockAcquire(&padesc->pa_mutex);
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ SpinLockRelease(&padesc->pa_mutex);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * goto_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * ----------------------------------------------------------------
+ */
+static int goto_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 25fd051..7ee0bb8 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -236,6 +236,7 @@ _copyAppend(const Append *from)
* copy remainder of node
*/
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 7418fbe..38ade5f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -369,6 +369,7 @@ _outAppend(StringInfo str, const Append *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index d3bbc02..fa1487a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1565,6 +1565,7 @@ _readAppend(void)
ReadCommonPlan(&local_node->plan);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 43bfd23..094809d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -99,7 +99,11 @@ static void generate_mergeappend_paths(PlannerInfo *root, RelOptInfo *rel,
static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
+static List *paths_insert_sorted_by_cost(List *paths, List *insert_paths);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1255,6 +1259,7 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
@@ -1277,14 +1282,42 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
*/
if (childrel->cheapest_total_path->param_info == NULL)
subpaths = accumulate_append_subpath(subpaths,
- childrel->cheapest_total_path);
+ childrel->cheapest_total_path);
else
subpaths_valid = false;
/* Same idea, but for a partial plan. */
if (childrel->partial_pathlist != NIL)
- partial_subpaths = accumulate_append_subpath(partial_subpaths,
- linitial(childrel->partial_pathlist));
+ partial_subpaths = accumulate_partialappend_subpath(
+ partial_subpaths,
+ linitial(childrel->partial_pathlist),
+ true, &nonpartial_subpaths);
+ else if (enable_parallelappend)
+ {
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ Path *parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* If we got one parallel-safe path, add it */
+ if (parallel_safe_path)
+ {
+ partial_subpaths =
+ accumulate_partialappend_subpath(partial_subpaths,
+ parallel_safe_path, false,
+ &nonpartial_subpaths);
+ }
+ else
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. So drop the idea for partial append path.
+ */
+ partial_subpaths_valid = false;
+ }
+ }
else
partial_subpaths_valid = false;
@@ -1359,7 +1392,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0));
/*
* Consider an append of partial unordered, unparameterized partial paths.
@@ -1367,26 +1401,14 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (partial_subpaths_valid)
{
AppendPath *appendpath;
- ListCell *lc;
- int parallel_workers = 0;
-
- /*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
- */
- foreach(lc, partial_subpaths)
- {
- Path *path = lfirst(lc);
+ int parallel_workers;
- parallel_workers = Max(parallel_workers, path->parallel_workers);
- }
- Assert(parallel_workers > 0);
+ parallel_workers = get_append_num_workers(partial_subpaths,
+ nonpartial_subpaths);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers);
+ appendpath = create_append_path(rel, nonpartial_subpaths,
+ partial_subpaths,
+ NULL, parallel_workers);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1438,7 +1460,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0));
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0));
}
}
@@ -1652,6 +1675,123 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * paths_insert_sorted_by_cost
+ *
+ * Insert each of the paths of 'insert_paths' into the right position in
+ * 'paths' so that 'paths' remains sorted by descending order of total cost.
+ *
+ * Return a possibly modified 'paths'.
+ */
+static List *
+paths_insert_sorted_by_cost(List *paths, List *insert_paths)
+{
+ ListCell *lci;
+
+ foreach(lci, insert_paths)
+ {
+ Path *insert_path = (Path *) lfirst(lci);
+ ListCell *lcp;
+ ListCell *prev;
+
+ /* Insert this 'insert_path' into 'paths' */
+ prev = NULL;
+ foreach(lcp, paths)
+ {
+ Path *path = (Path *) lfirst(lcp);
+
+ /* found the right position ? */
+ if (path->total_cost < insert_path->total_cost)
+ break;
+ prev = lcp;
+ }
+
+ /* Inserting before the first element ? */
+ if (prev == NULL)
+ paths = lcons(insert_path, paths);
+ else
+ (void) lappend_cell(paths, prev, insert_path);
+ }
+
+ return paths;
+}
+
+/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths =
+ paths_insert_sorted_by_cost(*nonpartial_subpaths,
+ apath_nonpartial_paths);
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths =
+ paths_insert_sorted_by_cost(*nonpartial_subpaths,
+ mpath->subpaths);
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths =
+ paths_insert_sorted_by_cost(*nonpartial_subpaths,
+ list_make1(subpath));
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1671,7 +1811,7 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL, 0));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..e8df075 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1704,6 +1705,98 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+ Cost highest_nonpartial_cost = 0;
+ int worker;
+
+ /*
+ * Make a note of the cost of first non-partial subpath, i.e. the first
+ * one in the list, if at all there are any non-partial subpaths.
+ */
+ if (num_nonpartial_subpaths > 0)
+ highest_nonpartial_cost = ((Path *) linitial(subpaths))->total_cost;
+
+ worker = 1;
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append. We don't consider non-partial paths separately. The
+ * parallel_divisor for non-partial paths is 1, and so overall we
+ * get a good approximation of per-worker cost.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (worker <= path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ worker++;
+ }
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ /*
+ * No matter how fast the partial plans finish, the Append path is
+ * going to take at least the time needed for the costliest non-partial
+ * path to finish. This is actually an approximation. We can even
+ * consider all the other non-partial plans and how workers would get
+ * scheduled to determine the cost of finishing the non-partial paths.
+ * But we anyway can't calculate the total cost exactly, especially
+ * because we can't determine exactly when some of the workers would
+ * start executing partial plans.
+ */
+ path->total_cost = Max(highest_nonpartial_cost, path->total_cost);
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 0551668..401d95a 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,7 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL, 0));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 89e1946..1702015 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,7 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist);
+ plan = make_append(subplans, best_path->first_partial_path, tlist);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5161,7 +5162,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist)
+make_append(List *appendplans, int first_partial_plan, List *tlist)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5171,6 +5172,7 @@ make_append(List *appendplans, List *tlist)
plan->lefttree = NULL;
plan->righttree = NULL;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 02286d9..529d91f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3345,10 +3345,7 @@ create_grouping_paths(PlannerInfo *root,
paths = lappend(paths, path);
}
path = (Path *)
- create_append_path(grouped_rel,
- paths,
- NULL,
- 0);
+ create_append_path(grouped_rel, paths, NIL, NULL, 0);
path->pathtarget = target;
}
else
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index 1389db1..e1d70a8 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,7 +566,7 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,7 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL, NULL, 0);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 8ce772d..6475e23 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -1193,6 +1193,70 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) i.e. ln(num_subpaths) / ln(2) */
+ log2w = log(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths))
+ / 0.693 ;
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,9 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer, int parallel_workers)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,40 +1276,29 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware =
+ (enable_parallelappend && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
- pathnode->subpaths = subpaths;
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 4feb26a..4f21c2e 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index b880dc1..79048af 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -228,6 +228,7 @@ typedef struct Append
{
Plan plan;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 05d6f07..eea8c72 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1108,15 +1108,22 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
Path path;
List *subpaths; /* list of component Paths */
+
+ /* Index of first partial path in subpaths */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 373c722..9622d2f 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,8 +64,11 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer, int parallel_workers);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6494b20..36be3a7 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1343,6 +1343,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1409,6 +1410,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index e3e9e34..810070a 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -463,11 +463,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Thu, Mar 16, 2017 at 3:57 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 12 March 2017 at 08:50, Robert Haas <robertmhaas@gmail.com> wrote:
However, Ashutosh's response made me think of something: one thing is
that we probably do want to group all of the non-partial plans at the
beginning of the Append so that they get workers first, and put the
partial plans afterward. That's because the partial plans can always
be accelerated by adding more workers as they become available, but
the non-partial plans are just going to take as long as they take - so
we want to start them as soon as possible. In fact, what we might
want to do is actually sort the non-partial paths in order of
decreasing cost, putting the most expensive one first and the others
in decreasing order after that - and then similarly afterward with the
partial paths. If we did that, we wouldn't need to store a bitmapset
OR two separate lists. We could just store the index of the first
partial plan in the list. Then you can test whether a path is partial
by checking whether this_index >= first_partial_index.Attached is an updated patch v7, which does the above. Now,
AppendState->subplans has all non-partial subplans followed by all
partial subplans, with the non-partial subplans in the order of
descending total cost. Also, for convenience, the AppendPath also now
has similar ordering in its AppendPath->subpaths. So there is a new
field both in Append and AppendPath : first_partial_path/plan, which
has value 0 if there are no non-partial subpaths.Also the backend now scans reverse, so that it does not take up the
most expensive path.There are also some changes in the costing done. Now that we know that
the very first path is the costliest non-partial path, we can use its
total cost as the total cost of Append in case all the partial path
costs are lesser.Modified/enhanced an existing test scenario in
src/test/regress/select_parallel.sql so that Parallel Append is
covered.As suggested by Robert, since pa_info->pa_finished was the only field
in pa_info, removed the ParallelAppendDescData.pa_info structure, and
instead brought pa_info->pa_finished into ParallelAppendDescData.+static inline void +exec_append_scan_first(AppendState *appendstate) +{ + appendstate->as_whichplan = 0; +}I don't think this is buying you anything, and suggest backing it out.
This is required for sequential Append, so that we can start executing
from the first subplan.My point is that there's really no point in defining a static inline
function containing one line of code. You could just put that line of
code in whatever places need it, which would probably be more clear.Did the same.
Some comments
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
From the comment, it looks like this condition will be encountered before the
backend returns any tuple. But this code is part of the loop which returns the
tuples. Shouldn't this be outside the loop? Why do we want to check a condition
for every row returned when the condition can happen only once and that too
before returning any tuple?
Why do we need following code in both ExecAppendInitializeWorker() and
ExecAppendInitializeDSM()? Both of those things happen before starting the
actual execution, so one of those should suffice?
+ /* Choose the optimal subplan to be executed. */
+ (void) parallel_append_next(node);
There is no pa_num_worker now, so probably this should get updated. Per comment
we should also get rid of SpinLockAcquire() and SpinLockRelease()?
+ * purpose. The spinlock is used so that it does not change the
+ * pa_num_workers field while workers are choosing the next node.
BTW, sa_finished seems to be a misnomor. The plan is not finished yet, but it
wants no more workers. So, should it be renamed as sa_no_new_workers or
something like that?
In parallel_append_next() we shouldn't need to call goto_next_plan() twice. If
the plan indicated by pa_next_plan is finished, all the plans must have
finished. This should be true if we set pa_next_plan to 0 at the time of
initialization. Any worker picking up pa_next_plan will set it to the next
valid plan. So the next worker asking for plan should pick pa_next_plan and
set it to the next one and so on.
I am wonding whether goto_next_plan() can be simplified as some module
arithmatic e.g. (whichplan - first_plan)++ % (last_plan - first_plan)
+ first_plan.
I am still reviewing the patch.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 16, 2017 at 8:48 AM, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
Why do we need following code in both ExecAppendInitializeWorker() and ExecAppendInitializeDSM()? Both of those things happen before starting the actual execution, so one of those should suffice? + /* Choose the optimal subplan to be executed. */ + (void) parallel_append_next(node);
ExecAppendInitializeWorker runs only in workers, but
ExecAppendInitializeDSM runs only in the leader.
BTW, sa_finished seems to be a misnomor. The plan is not finished yet, but it
wants no more workers. So, should it be renamed as sa_no_new_workers or
something like that?
I think that's not going to improve clarity. The comments can clarify
the exact semantics.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 16, 2017 at 6:27 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Attached is an updated patch v7, which does the above.
Some comments:
- You've added a GUC (which is good) but not documented it (which is
bad) or added it to postgresql.conf.sample (also bad).
- You've used a loop inside a spinlock-protected critical section,
which is against project policy. Use an LWLock; define and document a
new builtin tranche ID.
- The comment for pa_finished claims that it is the number of workers
executing the subplan, but it's a bool, not a count; I think this
comment is just out of date.
- paths_insert_sorted_by_cost() is a hand-coded insertion sort. Can't
we find a way to use qsort() for this instead of hand-coding a slower
algorithm? I think we could just create an array of the right length,
stick each path into it from add_paths_to_append_rel, and then qsort()
the array based on <is-partial, total-cost>. Then the result can be
turned into a list.
- Maybe the new helper functions in nodeAppend.c could get names
starting with exec_append_, to match the style of
exec_append_initialize_next().
- There's a superfluous whitespace change in add_paths_to_append_rel.
- The substantive changes in add_paths_to_append_rel don't look right
either. It's not clear why accumulate_partialappend_subpath is
getting called even in the non-enable_parallelappend case. I don't
think the logic for the case where we're not generating a parallel
append path needs to change at all.
- When parallel append is enabled, I think add_paths_to_append_rel
should still consider all the same paths that it does today, plus one
extra. The new path is a parallel append path where each subpath is
the cheapest subpath for that childrel, whether partial or
non-partial. If !enable_parallelappend, or if all of the cheapest
subpaths are partial, then skip this. (If all the cheapest subpaths
are non-partial, it's still potentially useful.) In other words,
don't skip consideration of parallel append just because you have a
partial path available for every child rel; it could be
- I think the way cost_append() works is not right. What you've got
assumes that you can just multiply the cost of a partial plan by the
parallel divisor to recover the total cost, which is not true because
we don't divide all elements of the plan cost by the parallel divisor
-- only the ones that seem like they should be divided. Also, it
could be smarter about what happens with the costs of non-partial
paths. I suggest the following algorithm instead.
1. Add up all the costs of the partial paths. Those contribute
directly to the final cost of the Append. This ignores the fact that
the Append may escalate the parallel degree, but I think we should
just ignore that problem for now, because we have no real way of
knowing what the impact of that is going to be.
2. Next, estimate the cost of the non-partial paths. To do this, make
an array of Cost of that length and initialize all the elements to
zero, then add the total cost of each non-partial plan in turn to the
element of the array with the smallest cost, and then take the maximum
of the array elements as the total cost of the non-partial plans. Add
this to the result from step 1 to get the total cost.
- In get_append_num_workers, instead of the complicated formula with
log() and 0.693, just add the list lengths and call fls() on the
result. Integer arithmetic FTW!
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 17 March 2017 at 01:37, Robert Haas <robertmhaas@gmail.com> wrote:
- You've added a GUC (which is good) but not documented it (which is
bad) or added it to postgresql.conf.sample (also bad).- You've used a loop inside a spinlock-protected critical section,
which is against project policy. Use an LWLock; define and document a
new builtin tranche ID.- The comment for pa_finished claims that it is the number of workers
executing the subplan, but it's a bool, not a count; I think this
comment is just out of date.
Yes, agreed. Will fix the above.
- paths_insert_sorted_by_cost() is a hand-coded insertion sort. Can't
we find a way to use qsort() for this instead of hand-coding a slower
algorithm? I think we could just create an array of the right length,
stick each path into it from add_paths_to_append_rel, and then qsort()
the array based on <is-partial, total-cost>. Then the result can be
turned into a list.
Yeah, I was in double minds as to whether to do the
copy-to-array-and-qsort thing, or should just write the same number of
lines of code to manually do an insertion sort. Actually I was
searching if we already have a linked list sort, but it seems we don't
have. Will do the qsort now since it would be faster.
- Maybe the new helper functions in nodeAppend.c could get names
starting with exec_append_, to match the style of
exec_append_initialize_next().- There's a superfluous whitespace change in add_paths_to_append_rel.
Will fix this.
- The substantive changes in add_paths_to_append_rel don't look right
either. It's not clear why accumulate_partialappend_subpath is
getting called even in the non-enable_parallelappend case. I don't
think the logic for the case where we're not generating a parallel
append path needs to change at all.
When accumulate_partialappend_subpath() is called for a childrel with
a partial path, it works just like accumulate_append_subpath() when
enable_parallelappend is false. That's why, for partial child path,
the same function is called irrespective of parallel-append or
non-parallel-append case. May be mentioning this in comments should
suffice here ?
- When parallel append is enabled, I think add_paths_to_append_rel
should still consider all the same paths that it does today, plus one
extra. The new path is a parallel append path where each subpath is
the cheapest subpath for that childrel, whether partial or
non-partial. If !enable_parallelappend, or if all of the cheapest
subpaths are partial, then skip this. (If all the cheapest subpaths
are non-partial, it's still potentially useful.)
In case of all-partial childrels, the paths are *exactly* same as
those that would have been created for enable_parallelappend=off. The
extra path is there for enable_parallelappend=on only when one or more
of the child rels do not have partial paths. Does this make sense ?
In other words,
don't skip consideration of parallel append just because you have a
partial path available for every child rel; it could be
I didn't get this. Are you saying that in the patch it is getting
skipped if enable_parallelappend = off ? I don't think so. For
all-partial child rels, partial append is always created. Only thing
is, in case of enable_parallelappend=off, the Append path is not
parallel_aware, so it executes just like it executes today under
Gather without being parallel-aware.
- I think the way cost_append() works is not right. What you've got
assumes that you can just multiply the cost of a partial plan by the
parallel divisor to recover the total cost, which is not true because
we don't divide all elements of the plan cost by the parallel divisor
-- only the ones that seem like they should be divided.
Yes, that was an approximation done. For those subpaths for which
there is no parallel_divsor, we cannot calculate the total cost
considering the number of workers for the subpath. I feel we should
consider the per-subpath parallel_workers somehow. The
Path->total_cost for a partial path is *always* per-worker cost, right
? Just want to confirm this assumption of mine.
Also, it
could be smarter about what happens with the costs of non-partial
paths. I suggest the following algorithm instead.1. Add up all the costs of the partial paths. Those contribute
directly to the final cost of the Append. This ignores the fact that
the Append may escalate the parallel degree, but I think we should
just ignore that problem for now, because we have no real way of
knowing what the impact of that is going to be.
I wanted to take into account per-subpath parallel_workers for total
cost of Append. Suppose the partial subpaths have per worker total
costs (3, 3, 3) and their parallel_workers are (2, 8, 4), with 2
Append workers available. So according to what you say, the total cost
is 9. With per-subplan parallel_workers taken into account, total cost
= (3*2 + 3*8 * 3*4)/2 = 21.
May be I didn't follow exactly what you suggested. Your logic is not
taking into account number of workers ? I am assuming you are
calculating per-worker total cost here.
2. Next, estimate the cost of the non-partial paths. To do this, make
an array of Cost of that length and initialize all the elements to
zero, then add the total cost of each non-partial plan in turn to the
element of the array with the smallest cost, and then take the maximum
of the array elements as the total cost of the non-partial plans. Add
this to the result from step 1 to get the total cost.
So with costs (8, 5, 2), add 8 and 5 to 2 so that it becomes (8, 5,
15) , and so the max is 15 ? I surely am misinterpreting this.
Actually, I couldn't come up with a general formula to find the
non-partial paths total cost, given the per-subplan cost and number of
workers. I mean, we can manually find out the total cost, but turning
it into a formula seems quite involved. We can even do a dry-run of
workers consuming each of the subplan slots and find the total time
time units taken, but finding some approximation seemed ok.
For e.g. we can manually find total time units taken for following :
costs (8, 2, 2, 2) with 2 workers : 8
costs (6, 6, 4, 1) with 2 workers : 10.
costs (6, 6, 4, 1) with 3 workers : 6.
But coming up with an alogrithm or a formula didn't look worth. So I
just did the total cost and divided it by workers. And besides that,
took the maximum of the 1st plan cost (since it is the highest) and
the average of total. I understand it would be too much approximation
for some cases, but another thing is, we don't know how to take into
account some of the workers shifting to partial workers. So the shift
may be quite fuzzy since all workers may not shift to partial plans
together.
- In get_append_num_workers, instead of the complicated formula with
log() and 0.693, just add the list lengths and call fls() on the
result. Integer arithmetic FTW!
Yeah fls() could be used. BTW I just found that costsize.c already has
this defined in the same way I did:
#define LOG2(x) (log(x) / 0.693147180559945)
May be we need to shift this to some common header file.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 16 March 2017 at 18:18, Ashutosh Bapat
<ashutosh.bapat@enterprisedb.com> wrote:
+ * Check if we are already finished plans from parallel append. This + * can happen if all the subplans are finished when this worker + * has not even started returning tuples. + */ + if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN) + return ExecClearTuple(node->ps.ps_ResultTupleSlot); From the comment, it looks like this condition will be encountered before the backend returns any tuple. But this code is part of the loop which returns the tuples. Shouldn't this be outside the loop? Why do we want to check a condition for every row returned when the condition can happen only once and that too before returning any tuple?
The way ExecProcNode() gets called, there is no different special code
that gets called instead of ExecProcNode() when a tuple is fetched for
the first time. I mean, we cannot prevent ExecProcNode() from getting
called when as_whichplan is invalid right from the beginning.
One thing we can do is : have a special slot in AppenState->as_plan[]
which has some dummy execution node that just returns NULL tuple, and
initially make as_whichplan point to this slot. But I think it is not
worth doing this.
We can instead reduce the if condition to:
if (node->as_whichplan == PA_INVALID_PLAN)
{
Assert(node->as_padesc != NULL);
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
}
BTW, the loop which you mentioned that returns tuples.... the loop is
not for returning tuples, the loop is for iterating to the next
subplan. Even if we take the condition out and keep it in the
beginning of ExecAppend, the issue will remain.
Why do we need following code in both ExecAppendInitializeWorker() and ExecAppendInitializeDSM()? Both of those things happen before starting the actual execution, so one of those should suffice? + /* Choose the optimal subplan to be executed. */ + (void) parallel_append_next(node);
ExecAppendInitializeWorker() is for the worker to attach (and then
initialize its own local data) to the dsm area created and shared by
ExecAppendInitializeDSM() in backend. But both worker and backend
needs to initialize its own as_whichplan to the next subplan.
There is no pa_num_worker now, so probably this should get updated. Per comment we should also get rid of SpinLockAcquire() and SpinLockRelease()? + * purpose. The spinlock is used so that it does not change the + * pa_num_workers field while workers are choosing the next node.
Will do this.
BTW, sa_finished seems to be a misnomor. The plan is not finished yet, but it
wants no more workers. So, should it be renamed as sa_no_new_workers or
something like that?
Actually in this context, "finished" means "we are done with this subplan".
In parallel_append_next() we shouldn't need to call goto_next_plan() twice. If
the plan indicated by pa_next_plan is finished, all the plans must have
finished. This should be true if we set pa_next_plan to 0 at the time of
initialization. Any worker picking up pa_next_plan will set it to the next
valid plan. So the next worker asking for plan should pick pa_next_plan and
set it to the next one and so on.
The current patch does not call it twice, but I might have overlooked
something. Let me know if I have.
I am wonding whether goto_next_plan() can be simplified as some module
arithmatic e.g. (whichplan - first_plan)++ % (last_plan - first_plan)
+ first_plan.
Hmm. IMHO it seems too much calculation for just shifting to next array element.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 17, 2017 at 10:12 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Yeah, I was in double minds as to whether to do the
copy-to-array-and-qsort thing, or should just write the same number of
lines of code to manually do an insertion sort. Actually I was
searching if we already have a linked list sort, but it seems we don't
have. Will do the qsort now since it would be faster.
relcache.c does an insertion sort with a list of OIDs. See insert_ordered_oid().
--
Peter Geoghegan
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
2. Next, estimate the cost of the non-partial paths. To do this, make
an array of Cost of that length and initialize all the elements to
zero, then add the total cost of each non-partial plan in turn to the
element of the array with the smallest cost, and then take the maximum
of the array elements as the total cost of the non-partial plans. Add
this to the result from step 1 to get the total cost.So with costs (8, 5, 2), add 8 and 5 to 2 so that it becomes (8, 5,
15) , and so the max is 15 ? I surely am misinterpreting this.Actually, I couldn't come up with a general formula to find the
non-partial paths total cost, given the per-subplan cost and number of
workers. I mean, we can manually find out the total cost, but turning
it into a formula seems quite involved. We can even do a dry-run of
workers consuming each of the subplan slots and find the total time
time units taken, but finding some approximation seemed ok.For e.g. we can manually find total time units taken for following :
costs (8, 2, 2, 2) with 2 workers : 8
costs (6, 6, 4, 1) with 2 workers : 10.
costs (6, 6, 4, 1) with 3 workers : 6.But coming up with an alogrithm or a formula didn't look worth. So I
just did the total cost and divided it by workers. And besides that,
took the maximum of the 1st plan cost (since it is the highest) and
the average of total. I understand it would be too much approximation
for some cases, but another thing is, we don't know how to take into
account some of the workers shifting to partial workers. So the shift
may be quite fuzzy since all workers may not shift to partial plans
together.
For non-partial paths, I did some comparison between the actual cost
and the cost taken by adding the per-subpath figures and dividing by
number of workers. And in the below cases, they do not differ
significantly. Here are the figures :
Case 1 :
Cost units of subpaths : 20 16 10 8 3 1.
Workers : 3
Actual total time to finish all workers : 20.
total/workers: 16.
Case 2 :
Cost units of subpaths : 20 16 10 8 3 1.
Workers : 2
Actual total time to finish all workers : 34.
total/workers: 32.
Case 3 :
Cost units of subpaths : 5 3 3 3 3
Workers : 3
Actual total time to finish all workers : 6
total/workers: 5.6
One more thing observed, is , in all of the above cases, all the
workers more or less finish at about the same time.
So this method seem to compare good which actual cost. The average
comes out a little less than the actual. But I think in the patch,
what I need to correct is, calculate separate per-worker costs of
non-partial and partial costs, and add them. This will give us
per-worker total cost, which is what a partial Append cost will be. I
just added all costs together.
There can be some extreme cases such as (5, 1, 1, 1, 1, 1) with 6
workers, where it will take at least 5 units, but average is 2. For
that we can clamp up the cost to the first path cost, so that for e.g.
it does not go lesser than 5 in this case.
Actually I have deviced one algorithm to calculate the exact time when
all workers finish non-partial costs. But I think it does not make
sense to apply it because it may be too much of calculation cost for
hundreds of paths.
But anyways, for archival purpose, here is the algorithm :
Per-subpath cost : 20 16 10 8 3 1, with 3 workers.
After 10 units (this is minimum of 20, 16, 10), the times remaining are :
10 6 0 8 3 1
After 6 units (minimum of 10, 06, 08), the times remaining are :
4 0 0 2 3 1
After 2 units (minimum of 4, 2, 3), the times remaining are :
2 0 0 0 1 1
After 1 units (minimum of 2, 1, 1), the times remaining are :
1 0 0 0 0 0
After 1 units (minimum of 1, 0 , 0), the times remaining are :
0 0 0 0 0 0
Now add up above time chunks : 10 + 6 + 2 + 1 + 1 = 20
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 17, 2017 at 1:12 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
- The substantive changes in add_paths_to_append_rel don't look right
either. It's not clear why accumulate_partialappend_subpath is
getting called even in the non-enable_parallelappend case. I don't
think the logic for the case where we're not generating a parallel
append path needs to change at all.When accumulate_partialappend_subpath() is called for a childrel with
a partial path, it works just like accumulate_append_subpath() when
enable_parallelappend is false. That's why, for partial child path,
the same function is called irrespective of parallel-append or
non-parallel-append case. May be mentioning this in comments should
suffice here ?
I don't get it. If you can get the same effect by changing something
or not changing it, presumably it'd be better to not change it. We
try not to change things just because we can; the change should be an
improvement in some way.
- When parallel append is enabled, I think add_paths_to_append_rel
should still consider all the same paths that it does today, plus one
extra. The new path is a parallel append path where each subpath is
the cheapest subpath for that childrel, whether partial or
non-partial. If !enable_parallelappend, or if all of the cheapest
subpaths are partial, then skip this. (If all the cheapest subpaths
are non-partial, it's still potentially useful.)In case of all-partial childrels, the paths are *exactly* same as
those that would have been created for enable_parallelappend=off. The
extra path is there for enable_parallelappend=on only when one or more
of the child rels do not have partial paths. Does this make sense ?
No, I don't think so. Imagine that we have three children, A, B, and
C. The cheapest partial paths have costs of 10,000 each. A, however,
has a non-partial path with a cost of 1,000. Even though A has a
partial path, we still want to consider a parallel append using the
non-partial path because it figures to be hugely faster.
The
Path->total_cost for a partial path is *always* per-worker cost, right
? Just want to confirm this assumption of mine.
Yes.
Also, it
could be smarter about what happens with the costs of non-partial
paths. I suggest the following algorithm instead.1. Add up all the costs of the partial paths. Those contribute
directly to the final cost of the Append. This ignores the fact that
the Append may escalate the parallel degree, but I think we should
just ignore that problem for now, because we have no real way of
knowing what the impact of that is going to be.I wanted to take into account per-subpath parallel_workers for total
cost of Append. Suppose the partial subpaths have per worker total
costs (3, 3, 3) and their parallel_workers are (2, 8, 4), with 2
Append workers available. So according to what you say, the total cost
is 9. With per-subplan parallel_workers taken into account, total cost
= (3*2 + 3*8 * 3*4)/2 = 21.
But that case never happens, because the parallel workers for the
append is always at least as large as the number of workers for any
single child.
May be I didn't follow exactly what you suggested. Your logic is not
taking into account number of workers ? I am assuming you are
calculating per-worker total cost here.2. Next, estimate the cost of the non-partial paths. To do this, make
an array of Cost of that length and initialize all the elements to
zero, then add the total cost of each non-partial plan in turn to the
element of the array with the smallest cost, and then take the maximum
of the array elements as the total cost of the non-partial plans. Add
this to the result from step 1 to get the total cost.So with costs (8, 5, 2), add 8 and 5 to 2 so that it becomes (8, 5,
15) , and so the max is 15 ? I surely am misinterpreting this.
No. If you have costs 8, 5, and 2 and only one process, cost is 15.
If you have two processes then for costing purposes you assume worker
1 will execute the first path (cost 8) and worker 2 will execute the
other two (cost 5 + 2 = 7), so the total cost is 8. If you have three
workers, the cost will still be 8, because there's no way to finish
the cost-8 path in less than 8 units of work.
- In get_append_num_workers, instead of the complicated formula with
log() and 0.693, just add the list lengths and call fls() on the
result. Integer arithmetic FTW!Yeah fls() could be used. BTW I just found that costsize.c already has
this defined in the same way I did:
#define LOG2(x) (log(x) / 0.693147180559945)
May be we need to shift this to some common header file.
LOG2() would make sense if you're working with a value represented as
a double, but if you have an integer input, I think fls() is better.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Attached is the updated patch that handles the changes for all the
comments except the cost changes part. Details about the specific
changes are after the cost-related points discussed below.
I wanted to take into account persubpath parallel_workers for total
cost of Append. Suppose the partial subpaths have per worker total
costs (3, 3, 3) and their parallel_workers are (2, 8, 4), with 2
Append workers available. So according to what you say, the total cost
is 9. With persubplan parallel_workers taken into account, total cost
= (3*2 + 3*8 * 3*4)/2 = 21.But that case never happens, because the parallel workers for the
append is always at least as large as the number of workers for any
single child.
Yeah, that's right. I will use this approach for partial paths.
For non-partial paths, I was checking following 3 options :
Option 1. Just take the sum of total non-partial child costs and
divide it by number of workers. It seems to be getting close to the
actual cost.
Option 2. Calculate exact cost by an algorithm which I mentioned
before, which is pasted below for reference :
Persubpath cost : 20 16 10 8 3 1, with 3 workers.
After 10 time units (this is minimum of first 3 i.e. 20, 16, 10), the
times remaining are :
10 6 0 8 3 1
After 6 units (minimum of 10, 06, 08), the times remaining are :
4 0 0 2 3 1
After 2 units (minimum of 4, 2, 3), the times remaining are :
2 0 0 0 1 1
After 1 units (minimum of 2, 1, 1), the times remaining are :
1 0 0 0 0 0
After 1 units (minimum of 1, 0 , 0), the times remaining are :
0 0 0 0 0 0
Now add up above time chunks : 10 + 6 + 2 + 1 + 1 = 20
Option 3. Get some approximation formula like you suggested. I am also
looking for such formula, just that some things are not clear to me.
The discussion of the same is below ...
2. Next, estimate the cost of the nonpartial paths. To do this, make
an array of Cost of that length and initialize all the elements to
zero, then add the total cost of each nonpartial plan in turn to the
element of the array with the smallest cost, and then take the maximum
of the array elements as the total cost of the nonpartial plans. Add
this to the result from step 1 to get the total cost.So with costs (8, 5, 2), add 8 and 5 to 2 so that it becomes (8, 5,
15) , and so the max is 15 ? I surely am misinterpreting this.No. If you have costs 8, 5, and 2 and only one process, cost is 15.
If you have two processes then for costing purposes you assume worker
1 will execute the first path (cost 8) and worker 2 will execute the
other two (cost 5 + 2 = 7), so the total cost is 8. If you have three
workers, the cost will still be 8, because there's no way to finish
the cost8 path in less than 8 units of work.
So the part that you suggested about adding up total cost in turn to
the smallest cost; this suggestion applies to only 1 worker right ?
For more than worker, are you suggesting to use some algorithm similar
to the one I suggested in option 2 above ? If yes, it would be great
if you again describe how that works for multiple workers. Or is it
that you were suggesting some simple approximate arithmetic that
applies to multiple workers ?
Like I mentioned, I will be happy to get such simple approximation
arithmetic that can be applied for multiple worker case. The one logic
I suggested in option 2 is something we can keep as the last option.
And option 1 is also an approximation but we would like to have a
better approximation. So wanted to clear my queries regarding option
3.
----------
Details about all the remaining changes in updated patch are below ...
On 20 March 2017 at 17:29, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Mar 17, 2017 at 1:12 PM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
- The substantive changes in add_paths_to_append_rel don't look right
either. It's not clear why accumulate_partialappend_subpath is
getting called even in the non-enable_parallelappend case. I don't
think the logic for the case where we're not generating a parallel
append path needs to change at all.When accumulate_partialappend_subpath() is called for a childrel with
a partial path, it works just like accumulate_append_subpath() when
enable_parallelappend is false. That's why, for partial child path,
the same function is called irrespective of parallel-append or
non-parallel-append case. May be mentioning this in comments should
suffice here ?I don't get it. If you can get the same effect by changing something
or not changing it, presumably it'd be better to not change it. We
try not to change things just because we can; the change should be an
improvement in some way.- When parallel append is enabled, I think add_paths_to_append_rel
should still consider all the same paths that it does today, plus one
extra. The new path is a parallel append path where each subpath is
the cheapest subpath for that childrel, whether partial or
non-partial. If !enable_parallelappend, or if all of the cheapest
subpaths are partial, then skip this. (If all the cheapest subpaths
are non-partial, it's still potentially useful.)In case of all-partial childrels, the paths are *exactly* same as
those that would have been created for enable_parallelappend=off. The
extra path is there for enable_parallelappend=on only when one or more
of the child rels do not have partial paths. Does this make sense ?No, I don't think so. Imagine that we have three children, A, B, and
C. The cheapest partial paths have costs of 10,000 each. A, however,
has a non-partial path with a cost of 1,000. Even though A has a
partial path, we still want to consider a parallel append using the
non-partial path because it figures to be hugely faster.
Right. Now that we want to consider both cheapest partial and cheapest
non-partial path, I now get what you were saying about having an extra
path for parallel_append. I have done all of the above changes. Now we
have an extra path for enable_parallelappend=true, besides the
non-parallel partial append path.
- You've added a GUC (which is good) but not documented it (which is
bad) or added it to postgresql.conf.sample (also bad).
Done.
- You've used a loop inside a spinlock-protected critical section,
which is against project policy. Use an LWLock; define and document a
new builtin tranche ID.
Done. Used LWlock for the parallel append synchronization. But I am
not sure what does "document the new builtin trancheID" mean. Didn't
find a readme which documents tranche ids.
For setting pa_finished=true when a partial plan finished, earlier it
was using Spinlock. Now it does not use any synchronization. It was
actually earlier using it because there was another field num_workers,
but it is not needed since there is no num_workers. I was considering
whether to use atomic read and write API in atomics.c for pa_finished.
But from what I understand, just a plain read/write is already atomic.
We require them only if there are some compound operations like
increment, exchange, etc.
- The comment for pa_finished claims that it is the number of workers
executing the subplan, but it's a bool, not a count; I think this
comment is just out of date.
Done.
- paths_insert_sorted_by_cost() is a hand-coded insertion sort. Can't
we find a way to use qsort() for this instead of hand-coding a slower
algorithm? I think we could just create an array of the right length,
stick each path into it from add_paths_to_append_rel, and then qsort()
the array based on <is-partial, total-cost>. Then the result can be
turned into a list.
Now added a new function list.c list_qsort() so that it can be used in
the future.
- Maybe the new helper functions in nodeAppend.c could get names
starting with exec_append_, to match the style of
exec_append_initialize_next().
Done.
- There's a superfluous whitespace change in add_paths_to_append_rel.
Didn't find exactly which, but I guess the attached latest patch does
not have it.
- In get_append_num_workers, instead of the complicated formula with
log() and 0.693, just add the list lengths and call fls() on the
result. Integer arithmetic FTW!Yeah fls() could be used. BTW I just found that costsize.c already has
this defined in the same way I did:
#define LOG2(x) (log(x) / 0.693147180559945)
May be we need to shift this to some common header file.LOG2() would make sense if you're working with a value represented as
a double, but if you have an integer input, I think fls() is better.
Used fls() now.
Attachments:
ParallelAppend_v8.patchapplication/octet-stream; name=ParallelAppend_v8.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b379b67..a8e3737 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3618,6 +3618,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86db73b..2ba9472 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..1ffa803 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,10 +59,48 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
-static bool exec_append_initialize_next(AppendState *appendstate);
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_goto_next_plan(int curplan, int first_plan,
+ int last_plan);
+
/* ----------------------------------------------------------------
* exec_append_initialize_next
*
@@ -77,6 +115,27 @@ exec_append_initialize_next(AppendState *appendstate)
int whichplan;
/*
+ * In case it's parallel-aware, follow it's own logic of choosing the next
+ * subplan.
+ */
+ if (appendstate->as_padesc)
+ {
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(appendstate->ps.state->es_direction));
+
+ return exec_append_parallel_next(appendstate);
+ }
+
+ /*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +241,9 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * In case it's a sequential Append, initialize to scan first subplan.
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -205,6 +263,14 @@ ExecAppend(AppendState *node)
TupleTableSlot *result;
/*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
+ /*
* figure out which subplan we are currently processing
*/
subnode = node->appendplans[node->as_whichplan];
@@ -225,14 +291,18 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * We are done with this subplan. There might be other workers still
+ * processing the last chunk of rows for this same subplan, but there's
+ * no point for new workers to run this subplan, so mark this subplan
+ * as finished.
+ */
+ if (node->as_padesc)
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ /*
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
- else
- node->as_whichplan--;
if (!exec_append_initialize_next(node))
return ExecClearTuple(node->ps.ps_ResultTupleSlot);
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,264 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ /*
+ * Just setting all the number of workers to 0 is enough. The logic
+ * of choosing the next plan in workers will take care of everything
+ * else.
+ */
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
+
+ /* Choose the optimal subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the optimal subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_goto_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ /* If we didn't find any plan to execute, stop executing. */
+ if (!found)
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_goto_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_goto_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * ----------------------------------------------------------------
+ */
+static int exec_append_goto_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 67c7de6..873c955 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1b9005f..7b22ca5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 474f221..44da33a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1568,6 +1568,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a1e1a87..6611e45 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1264,7 +1267,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1300,7 +1307,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1378,23 +1443,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1404,9 +1485,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1459,7 +1540,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1677,6 +1759,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1696,7 +1850,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..e8df075 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1704,6 +1705,98 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+ Cost highest_nonpartial_cost = 0;
+ int worker;
+
+ /*
+ * Make a note of the cost of first non-partial subpath, i.e. the first
+ * one in the list, if at all there are any non-partial subpaths.
+ */
+ if (num_nonpartial_subpaths > 0)
+ highest_nonpartial_cost = ((Path *) linitial(subpaths))->total_cost;
+
+ worker = 1;
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append. We don't consider non-partial paths separately. The
+ * parallel_divisor for non-partial paths is 1, and so overall we
+ * get a good approximation of per-worker cost.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (worker <= path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ worker++;
+ }
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ /*
+ * No matter how fast the partial plans finish, the Append path is
+ * going to take at least the time needed for the costliest non-partial
+ * path to finish. This is actually an approximation. We can even
+ * consider all the other non-partial plans and how workers would get
+ * scheduled to determine the cost of finishing the non-partial paths.
+ * But we anyway can't calculate the total cost exactly, especially
+ * because we can't determine exactly when some of the workers would
+ * start executing partial plans.
+ */
+ path->total_cost = Max(highest_nonpartial_cost, path->total_cost);
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c80c999..c517900 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5163,7 +5165,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5174,6 +5176,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index cbdea1f..9dd4ef3 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3383,8 +3383,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d88738e..4069855 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fca96eb..9f962e0 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,69 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1278,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..36b8750 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 4feb26a..4f21c2e 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index a02b154..5383509 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 4a95e16..1950192 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1c88a79..70ccdbf 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1112,10 +1112,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1123,6 +1127,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81640de..2203ab4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Wed, Mar 22, 2017 at 4:49 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Attached is the updated patch that handles the changes for all the
comments except the cost changes part. Details about the specific
changes are after the cost-related points discussed below.For non-partial paths, I was checking following 3 options :
Option 1. Just take the sum of total non-partial child costs and
divide it by number of workers. It seems to be getting close to the
actual cost.
If the costs for all children are about equal, then that works fine.
But when they are very unequal, then it's highly misleading.
Option 2. Calculate exact cost by an algorithm which I mentioned
before, which is pasted below for reference :
Persubpath cost : 20 16 10 8 3 1, with 3 workers.
After 10 time units (this is minimum of first 3 i.e. 20, 16, 10), the
times remaining are :
10 6 0 8 3 1
After 6 units (minimum of 10, 06, 08), the times remaining are :
4 0 0 2 3 1
After 2 units (minimum of 4, 2, 3), the times remaining are :
2 0 0 0 1 1
After 1 units (minimum of 2, 1, 1), the times remaining are :
1 0 0 0 0 0
After 1 units (minimum of 1, 0 , 0), the times remaining are :
0 0 0 0 0 0
Now add up above time chunks : 10 + 6 + 2 + 1 + 1 = 20
This gives the same answer as what I was proposing, but I believe it's
more complicated to compute. The way my proposal would work in this
case is that we would start with an array C[3] (since there are three
workers], with all entries 0. Logically C[i] represents the amount of
work to be performed by worker i. We add each path in turn to the
worker whose array entry is currently smallest; in the case of a tie,
just pick the first such entry.
So in your example we do this:
C[0] += 20;
C[1] += 16;
C[2] += 10;
/* C[2] is smaller than C[0] or C[1] at this point, so we add the next
path to C[2] */
C[2] += 8;
/* after the previous line, C[1] is now the smallest, so add to that
entry next */
C[1] += 3;
/* now we've got C[0] = 20, C[1] = 19, C[2] = 18, so add to C[2] */
C[2] += 1;
/* final result: C[0] = 20, C[1] = 19, C[2] = 19 */
Now we just take the highest entry that appears in any array, which in
this case is C[0], as the total cost.
Comments on this latest version:
In my previous review, I said that you should "define and document a
new builtin tranche ID"; you did the first but not the second. See
the table in monitoring.sgml.
Definition of exec_append_goto_next_plan should have a line break
after the return type, per usual PostgreSQL style rules.
- * initialize to scan first subplan
+ * In case it's a sequential Append, initialize to scan first subplan.
This comment is confusing because the code is executed whether it's
parallel or not. I think it might be better to write something like
"initialize to scan first subplan (but note that we'll override this
later in the case of a parallel append)"
/*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
There seems to be no reason why this couldn't be hoisted out of the
loop. Actually, I think Ashutosh pointed this out before, but I
didn't understand at that time what his point was. Looking back, I
see that he also pointed out that the as_padesc test isn't necessary,
which is also true.
+ if (node->as_padesc)
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
I think you should move this logic inside exec_append_parallel_next.
That would avoid testing node->pa_desc an extra time for non-parallel
append. I note that the comment doesn't explain why it's safe to do
this without taking the lock. I think we could consider doing it with
the lock held, but it probably is safe, because we're only setting it
from false to true. If someone else does the same thing, that won't
hurt anything, and if someone else fails to see our update, then the
worst-case scenario is that they'll try to execute a plan that has no
chance of returning any more rows. That's not so bad. Actually,
looking further, you do have a comment explaining that, but it's in
exec_append_parallel_next() where the value is used, rather than here.
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+ node->as_padesc = padesc;
Putting the shm_toc_insert call after we fully initialize the
structure seems better than putting it after we've done some of the
initialization and before we've done the rest.
+ /* Choose the optimal subplan to be executed. */
I think the word "first" would be more accurate than "optimal". We
can only hope to pick the optimal one, but whichever one we pick is
definitely the one we're executing first!
I think the loop in exec_append_parallel_next() is a bit confusing.
It has three exit conditions, one checked at the top of the loop and
two other ways to exit via break statements. Sometimes it exits
because whichplan == PA_INVALID_PLAN was set by
exec_append_goto_next_plan(), and other times it exits because
whichplan == initial_plan and then it sets whichplan ==
PA_INVALID_PLAN itself. I feel like this whole function could be
written more simply somehow.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 23 March 2017 at 05:55, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Mar 22, 2017 at 4:49 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Attached is the updated patch that handles the changes for all the
comments except the cost changes part. Details about the specific
changes are after the cost-related points discussed below.For non-partial paths, I was checking following 3 options :
Option 1. Just take the sum of total non-partial child costs and
divide it by number of workers. It seems to be getting close to the
actual cost.If the costs for all children are about equal, then that works fine.
But when they are very unequal, then it's highly misleading.Option 2. Calculate exact cost by an algorithm which I mentioned
before, which is pasted below for reference :
Persubpath cost : 20 16 10 8 3 1, with 3 workers.
After 10 time units (this is minimum of first 3 i.e. 20, 16, 10), the
times remaining are :
10 6 0 8 3 1
After 6 units (minimum of 10, 06, 08), the times remaining are :
4 0 0 2 3 1
After 2 units (minimum of 4, 2, 3), the times remaining are :
2 0 0 0 1 1
After 1 units (minimum of 2, 1, 1), the times remaining are :
1 0 0 0 0 0
After 1 units (minimum of 1, 0 , 0), the times remaining are :
0 0 0 0 0 0
Now add up above time chunks : 10 + 6 + 2 + 1 + 1 = 20
This gives the same answer as what I was proposing
Ah I see.
but I believe it's more complicated to compute.
Yes a bit, particularly because in my algorithm, I would have to do
'n' subtractions each time, in case of 'n' workers. But it looked more
natural because it follows exactly the way we manually calculate.
The way my proposal would work in this
case is that we would start with an array C[3] (since there are three
workers], with all entries 0. Logically C[i] represents the amount of
work to be performed by worker i. We add each path in turn to the
worker whose array entry is currently smallest; in the case of a tie,
just pick the first such entry.So in your example we do this:
C[0] += 20;
C[1] += 16;
C[2] += 10;
/* C[2] is smaller than C[0] or C[1] at this point, so we add the next
path to C[2] */
C[2] += 8;
/* after the previous line, C[1] is now the smallest, so add to that
entry next */
C[1] += 3;
/* now we've got C[0] = 20, C[1] = 19, C[2] = 18, so add to C[2] */
C[2] += 1;
/* final result: C[0] = 20, C[1] = 19, C[2] = 19 */Now we just take the highest entry that appears in any array, which in
this case is C[0], as the total cost.
Wow. The way your final result exactly tallies with my algorithm
result is very interesting. This looks like some maths or computer
science theory that I am not aware.
I am currently coding the algorithm using your method. Meanwhile
attached is a patch that takes care of your other comments, details of
which are below...
In my previous review, I said that you should "define and document a
new builtin tranche ID"; you did the first but not the second. See
the table in monitoring.sgml.
Yeah, I tried to search how TBM did in the source, but I guess I
didn't correctly run "git grep" commands, so the results did not have
monitoring.sgml, so I thought may be you mean something else by
"document".
Added changes in monitoring.sgml now.
Definition of exec_append_goto_next_plan should have a line break
after the return type, per usual PostgreSQL style rules.
Oops. Done.
- * initialize to scan first subplan + * In case it's a sequential Append, initialize to scan first subplan.This comment is confusing because the code is executed whether it's
parallel or not. I think it might be better to write something like
"initialize to scan first subplan (but note that we'll override this
later in the case of a parallel append)"
Done.
/* + * Check if we are already finished plans from parallel append. This + * can happen if all the subplans are finished when this worker + * has not even started returning tuples. + */ + if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN) + return ExecClearTuple(node->ps.ps_ResultTupleSlot);There seems to be no reason why this couldn't be hoisted out of the
loop. Actually, I think Ashutosh pointed this out before, but I
didn't understand at that time what his point was. Looking back, I
see that he also pointed out that the as_padesc test isn't necessary,
which is also true.
I am assuming both yours and Ashutosh's concern is that this check
will be executed for *each* tuple returned, and which needs to be
avoided. Actually, just moving it out of the loop is not going to
solve the runs-for-each-tuple issue. It still will execute for each
tuple. But after a thought, now I agree this can be taken out of loop
anyways, but, not for solving the per-tuple issue, but because it need
not be run for each of the iteration of the loop because that loop is
there to go to the next subplan.
When a worker tries to choose a plan to execute at the very beginning
(i.e in ExecAppendInitializeWorker()), it sometimes finds there is no
plan to execute, because all the others have already taken them and
they are already finished or they are all non-partial plans. In short,
for all subplans, pa_finished = true. So as_whichplan has to be
PA_INVALID_PLAN. To get rid of the extra check in ExecAppend(), in
ExecAppendInitializeWorker() if all plans are finished, we can very
well assign as_whichplan to a partial plan which has already finished,
so that ExecAppend() will execute this finished subplan and just
return NULL. But if all plans are non-partial, we cannot do that.
Now, when ExecAppend() is called, there is no way to know whether this
is the first time ExecProcNode() is executed or not. So we have to
keep on checking the node->as_whichplan == PA_INVALID_PLAN condition.
My earlier response to Ashutosh's feedback on this same point are
pasted below, where there are some possible improvements discussed :
The way ExecProcNode() gets called, there is no different special code
that gets called instead of ExecProcNode() when a tuple is fetched for
the first time. I mean, we cannot prevent ExecProcNode() from getting
called when as_whichplan is invalid right from the beginning.
One thing we can do is : have a special slot in AppenState>as_plan[]
which has some dummy execution node that just returns NULL tuple, and
initially make as_whichplan point to this slot. But I think it is not
worth doing this.
We can instead reduce the if condition to:
if (node>as_whichplan == PA_INVALID_PLAN)
{
Assert(node>as_padesc != NULL);
return ExecClearTuple(node>ps.ps_ResultTupleSlot);
}
BTW, the loop which you mentioned that returns tuples.... the loop is
not for returning tuples, the loop is for iterating to the next
subplan. Even if we take the condition out and keep it in the
beginning of ExecAppend, the issue will remain.
+ if (node->as_padesc) + node->as_padesc->pa_finished[node->as_whichplan] = true;I think you should move this logic inside exec_append_parallel_next.
That would avoid testing node->pa_desc an extra time for non-parallel
append.
Actually exec_append_parallel_next() is called at other places also,
for which we cannot set pa_finished to true inside
exec_append_parallel_next().
But I have done the changes another way. I have taken
exec_append_parallel_next() out of exec_append_initialize_next(), and
put two different conditional code blocks in ExecAppend(), one which
calls set_finished() followed by exec_append_parallel_next() and the
other calls exec_append_initialize_next() (now renamed to
exec_append_seq_next()
But one thing to note is that this condition is not executed for each
tuple. It is only while going to the next subplan.
I note that the comment doesn't explain why it's safe to do
this without taking the lock. I think we could consider doing it with
the lock held, but it probably is safe, because we're only setting it
from false to true. If someone else does the same thing, that won't
hurt anything, and if someone else fails to see our update, then the
worst-case scenario is that they'll try to execute a plan that has no
chance of returning any more rows. That's not so bad. Actually,
looking further, you do have a comment explaining that, but it's in
exec_append_parallel_next() where the value is used, rather than here.
Yes, right.
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans); + + shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc); + node->as_padesc = padesc;Putting the shm_toc_insert call after we fully initialize the
structure seems better than putting it after we've done some of the
initialization and before we've done the rest.
Done. Also found out that I was memset()ing only pa_finished[]. Now
there is a memset for the whole ParallelAppendDesc structure.
+ /* Choose the optimal subplan to be executed. */
I think the word "first" would be more accurate than "optimal". We
can only hope to pick the optimal one, but whichever one we pick is
definitely the one we're executing first!
Done.
I think the loop in exec_append_parallel_next() is a bit confusing.
It has three exit conditions, one checked at the top of the loop and
two other ways to exit via break statements. Sometimes it exits
because whichplan == PA_INVALID_PLAN was set by
exec_append_goto_next_plan(), and other times it exits because
whichplan == initial_plan
Yeah, we cannot bring up the (whichplan == initialplan) to the top in
for(;;) because initially whichplan is initialplan, and we have to
execute the loop at least once (unless whichplan = INVALID).
And we cannot bring down the for condition (which != PA_INVALID_PLAN)
because whichplan can be INVALID right at the beginning if
pa_next_plan itself can be PA_INVALID_PLAN.
and then it sets whichplan == PA_INVALID_PLAN itself.
It sets that to PA_INVALID_PLAN only when it does not find any next
plan to execute. This is essential to do that particularly because
initiallly when ExecAppendInitialize[Worker/DSM]() is called, it
cannot set as_whichplan to any valid value.
I feel like this whole function could be written more simply somehow.
Yeah, the main reason it is a bit compilcated is because we are
simulating circular array structure, and that too with an optimization
that we can skip the finished non-partial plans while wrapping around
to the next plan in the circular array. I have tried to add a couple
of more comments.
Also renamed exec_append_goto_next_plan() to
exec_append_get_next_plan() since it is not actually shifting any
counter, it is just returning what is the next counter.
Attachments:
ParallelAppend_v9.patchapplication/octet-stream; name=ParallelAppend_v9.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b379b67..a8e3737 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3618,6 +3618,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index dcb2d33..49a053a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -832,7 +832,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1092,6 +1092,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 86db73b..2ba9472 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..e9e8676 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 67c7de6..873c955 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1b9005f..7b22ca5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 474f221..44da33a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1568,6 +1568,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a1e1a87..6611e45 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1264,7 +1267,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1300,7 +1307,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1378,23 +1443,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1404,9 +1485,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1459,7 +1540,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1677,6 +1759,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1696,7 +1850,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..e8df075 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -1704,6 +1705,98 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+ Cost highest_nonpartial_cost = 0;
+ int worker;
+
+ /*
+ * Make a note of the cost of first non-partial subpath, i.e. the first
+ * one in the list, if at all there are any non-partial subpaths.
+ */
+ if (num_nonpartial_subpaths > 0)
+ highest_nonpartial_cost = ((Path *) linitial(subpaths))->total_cost;
+
+ worker = 1;
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append. We don't consider non-partial paths separately. The
+ * parallel_divisor for non-partial paths is 1, and so overall we
+ * get a good approximation of per-worker cost.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (worker <= path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ worker++;
+ }
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ /*
+ * No matter how fast the partial plans finish, the Append path is
+ * going to take at least the time needed for the costliest non-partial
+ * path to finish. This is actually an approximation. We can even
+ * consider all the other non-partial plans and how workers would get
+ * scheduled to determine the cost of finishing the non-partial paths.
+ * But we anyway can't calculate the total cost exactly, especially
+ * because we can't determine exactly when some of the workers would
+ * start executing partial plans.
+ */
+ path->total_cost = Max(highest_nonpartial_cost, path->total_cost);
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c80c999..c517900 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5163,7 +5165,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5174,6 +5176,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 68d74cb..1529396 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3383,8 +3383,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d88738e..4069855 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fca96eb..9f962e0 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,69 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1278,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..36b8750 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 4feb26a..4f21c2e 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index a02b154..5383509 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 4a95e16..1950192 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1c88a79..70ccdbf 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1112,10 +1112,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1123,6 +1127,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81640de..2203ab4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On 23 March 2017 at 16:26, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 23 March 2017 at 05:55, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Mar 22, 2017 at 4:49 AM, Amit Khandekar <amitdkhan.pg@gmail.com>
wrote:
Attached is the updated patch that handles the changes for all the
comments except the cost changes part. Details about the specific
changes are after the cost-related points discussed below.For non-partial paths, I was checking following 3 options :
Option 1. Just take the sum of total non-partial child costs and
divide it by number of workers. It seems to be getting close to the
actual cost.If the costs for all children are about equal, then that works fine.
But when they are very unequal, then it's highly misleading.Option 2. Calculate exact cost by an algorithm which I mentioned
before, which is pasted below for reference :
Persubpath cost : 20 16 10 8 3 1, with 3 workers.
After 10 time units (this is minimum of first 3 i.e. 20, 16, 10), the
times remaining are :
10 6 0 8 3 1
After 6 units (minimum of 10, 06, 08), the times remaining are :
4 0 0 2 3 1
After 2 units (minimum of 4, 2, 3), the times remaining are :
2 0 0 0 1 1
After 1 units (minimum of 2, 1, 1), the times remaining are :
1 0 0 0 0 0
After 1 units (minimum of 1, 0 , 0), the times remaining are :
0 0 0 0 0 0
Now add up above time chunks : 10 + 6 + 2 + 1 + 1 = 20This gives the same answer as what I was proposing
Ah I see.
but I believe it's more complicated to compute.
Yes a bit, particularly because in my algorithm, I would have to do
'n' subtractions each time, in case of 'n' workers. But it looked more
natural because it follows exactly the way we manually calculate.The way my proposal would work in this
case is that we would start with an array C[3] (since there are three
workers], with all entries 0. Logically C[i] represents the amount of
work to be performed by worker i. We add each path in turn to the
worker whose array entry is currently smallest; in the case of a tie,
just pick the first such entry.So in your example we do this:
C[0] += 20;
C[1] += 16;
C[2] += 10;
/* C[2] is smaller than C[0] or C[1] at this point, so we add the next
path to C[2] */
C[2] += 8;
/* after the previous line, C[1] is now the smallest, so add to that
entry next */
C[1] += 3;
/* now we've got C[0] = 20, C[1] = 19, C[2] = 18, so add to C[2] */
C[2] += 1;
/* final result: C[0] = 20, C[1] = 19, C[2] = 19 */Now we just take the highest entry that appears in any array, which in
this case is C[0], as the total cost.Wow. The way your final result exactly tallies with my algorithm
result is very interesting. This looks like some maths or computer
science theory that I am not aware.I am currently coding the algorithm using your method.
While I was coding this, I was considering if Path->rows also should
be calculated similar to total cost for non-partial subpath and total
cost for partial subpaths. I think for rows, we can just take
total_rows divided by workers for non-partial paths, and this
approximation should suffice. It looks odd that it be treated with the
same algorithm we chose for total cost for non-partial paths.
Meanwhile, attached is a WIP patch v10. The only change in this patch
w.r.t. the last patch (v9) is that this one has a new function defined
append_nonpartial_cost(). Just sending this to show how the algorithm
looks like; haven't yet called it.
Attachments:
ParallelAppend_v10_WIP.patchapplication/octet-stream; name=ParallelAppend_v10_WIP.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 2de3540..a7aad08 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3643,6 +3643,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e930731..6f51372 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -832,7 +832,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1096,6 +1096,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index b91b663..8b0ec2c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..e9e8676 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 93d4eb2..60f0b7e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1b9005f..7b22ca5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 474f221..44da33a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1568,6 +1568,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a1e1a87..6611e45 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1264,7 +1267,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1300,7 +1307,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1378,23 +1443,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1404,9 +1485,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1459,7 +1540,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1677,6 +1759,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1696,7 +1850,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..9eff4b9 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(Path *path, List *subpaths);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1704,6 +1706,161 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ * subpaths contains only non-partial subpaths.
+ */
+static Cost
+append_nonpartial_cost(Path *path, List *subpaths)
+{
+ Cost *costarr;
+ int len = path->parallel_workers;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int min_index;
+ int max_index;
+
+ /* Build the cost array out of first 'parallel_workers' elements of subpaths */
+ costarr = (Cost *) palloc(sizeof(Cost) * len);
+ i = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+ if (i == len)
+ break;
+ costarr[i++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = len - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < len; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+
+ for (max_index = i = 0; i < len; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+ Cost highest_nonpartial_cost = 0;
+ int worker;
+
+ /*
+ * Make a note of the cost of first non-partial subpath, i.e. the first
+ * one in the list, if at all there are any non-partial subpaths.
+ */
+ if (num_nonpartial_subpaths > 0)
+ highest_nonpartial_cost = ((Path *) linitial(subpaths))->total_cost;
+
+ worker = 1;
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append. We don't consider non-partial paths separately. The
+ * parallel_divisor for non-partial paths is 1, and so overall we
+ * get a good approximation of per-worker cost.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (worker <= path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ worker++;
+ }
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ /*
+ * No matter how fast the partial plans finish, the Append path is
+ * going to take at least the time needed for the costliest non-partial
+ * path to finish. This is actually an approximation. We can even
+ * consider all the other non-partial plans and how workers would get
+ * scheduled to determine the cost of finishing the non-partial paths.
+ * But we anyway can't calculate the total cost exactly, especially
+ * because we can't determine exactly when some of the workers would
+ * start executing partial plans.
+ */
+ path->total_cost = Max(highest_nonpartial_cost, path->total_cost);
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c80c999..c517900 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5163,7 +5165,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5174,6 +5176,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 68d74cb..1529396 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3383,8 +3383,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d88738e..4069855 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fca96eb..9f962e0 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,69 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1278,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..36b8750 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 291bf76..3942e8a 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index a02b154..5383509 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 4a95e16..1950192 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1c88a79..70ccdbf 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1112,10 +1112,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1123,6 +1127,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81640de..2203ab4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Fri, Mar 24, 2017 at 12:38 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
Meanwhile, attached is a WIP patch v10. The only change in this patch
w.r.t. the last patch (v9) is that this one has a new function defined
append_nonpartial_cost(). Just sending this to show how the algorithm
looks like; haven't yet called it.
Hi,
I have given patch on latest pg sources (on commit
457a4448732881b5008f7a3bcca76fc299075ac3). configure and make all
install ran successfully, but initdb failed with below error.
[edb@localhost bin]$ ./initdb -D data
The files belonging to this database system will be owned by user "edb".
This user must also own the server process.
The database cluster will be initialized with locale "en_US.UTF-8".
The default database encoding has accordingly been set to "UTF8".
The default text search configuration will be set to "english".
Data page checksums are disabled.
creating directory data ... ok
creating subdirectories ... ok
selecting default max_connections ... sh: line 1: 3106 Aborted
(core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=100 -c shared_buffers=1000 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
sh: line 1: 3112 Aborted (core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=50 -c shared_buffers=500 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
sh: line 1: 3115 Aborted (core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=40 -c shared_buffers=400 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
sh: line 1: 3118 Aborted (core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=30 -c shared_buffers=300 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
sh: line 1: 3121 Aborted (core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=20 -c shared_buffers=200 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
sh: line 1: 3124 Aborted (core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=10 -c shared_buffers=100 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
10
selecting default shared_buffers ... sh: line 1: 3127 Aborted
(core dumped)
"/home/edb/WORKDB/PG3/postgresql/inst/bin/postgres" --boot -x0 -F -c
max_connections=10 -c shared_buffers=16384 -c
dynamic_shared_memory_type=none < "/dev/null" > "/dev/null" 2>&1
400kB
selecting dynamic shared memory implementation ... posix
creating configuration files ... ok
running bootstrap script ... TRAP:
FailedAssertion("!(LWLockTranchesAllocated >=
LWTRANCHE_FIRST_USER_DEFINED)", File: "lwlock.c", Line: 501)
child process was terminated by signal 6: Aborted
initdb: removing data directory "data"
[edb@localhost bin]$
Thanks & Regards,
Rajkumar Raghuwanshi
QMG, EnterpriseDB Corporation
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 24 March 2017 at 13:11, Rajkumar Raghuwanshi
<rajkumar.raghuwanshi@enterprisedb.com> wrote:
I have given patch on latest pg sources (on commit
457a4448732881b5008f7a3bcca76fc299075ac3). configure and make all
install ran successfully, but initdb failed with below error.
FailedAssertion("!(LWLockTranchesAllocated >=
LWTRANCHE_FIRST_USER_DEFINED)", File: "lwlock.c", Line: 501)
Thanks for reporting, Rajkumar.
With the new PARALLEL_APPEND tranche ID, LWTRANCHE_FIRST_USER_DEFINED
value has crossed the value 64. So we need to increase the initial
size of LWLockTrancheArray from 64 to 128. Attached is the updated
patch.
Attachments:
ParallelAppend_v11.patchapplication/octet-stream; name=ParallelAppend_v11.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 2de3540..a7aad08 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3643,6 +3643,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e930731..6f51372 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -832,7 +832,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1096,6 +1096,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index b91b663..8b0ec2c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..e9e8676 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 93d4eb2..60f0b7e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1b9005f..7b22ca5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 474f221..44da33a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1568,6 +1568,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a1e1a87..6611e45 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1264,7 +1267,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1300,7 +1307,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1378,23 +1443,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1404,9 +1485,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1459,7 +1540,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1677,6 +1759,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1696,7 +1850,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..9eff4b9 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,7 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(Path *path, List *subpaths);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1704,6 +1706,161 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ * subpaths contains only non-partial subpaths.
+ */
+static Cost
+append_nonpartial_cost(Path *path, List *subpaths)
+{
+ Cost *costarr;
+ int len = path->parallel_workers;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int min_index;
+ int max_index;
+
+ /* Build the cost array out of first 'parallel_workers' elements of subpaths */
+ costarr = (Cost *) palloc(sizeof(Cost) * len);
+ i = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+ if (i == len)
+ break;
+ costarr[i++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = len - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < len; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+
+ for (max_index = i = 0; i < len; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (path->parallel_aware)
+ {
+ int parallel_divisor;
+ Cost highest_nonpartial_cost = 0;
+ int worker;
+
+ /*
+ * Make a note of the cost of first non-partial subpath, i.e. the first
+ * one in the list, if at all there are any non-partial subpaths.
+ */
+ if (num_nonpartial_subpaths > 0)
+ highest_nonpartial_cost = ((Path *) linitial(subpaths))->total_cost;
+
+ worker = 1;
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * The subpath rows and cost is per worker. We need total count
+ * of each of the subpaths, so that we can determine the total cost
+ * of Append. We don't consider non-partial paths separately. The
+ * parallel_divisor for non-partial paths is 1, and so overall we
+ * get a good approximation of per-worker cost.
+ */
+ parallel_divisor = get_parallel_divisor(subpath);
+ path->rows += (subpath->rows * parallel_divisor);
+ path->total_cost += (subpath->total_cost * parallel_divisor);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (worker <= path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ worker++;
+ }
+ }
+
+ /* The row count and cost should represent per-worker figures. */
+ parallel_divisor = get_parallel_divisor(path);
+ path->rows = clamp_row_est(path->rows / parallel_divisor);
+ path->total_cost /= parallel_divisor;
+
+ /*
+ * No matter how fast the partial plans finish, the Append path is
+ * going to take at least the time needed for the costliest non-partial
+ * path to finish. This is actually an approximation. We can even
+ * consider all the other non-partial plans and how workers would get
+ * scheduled to determine the cost of finishing the non-partial paths.
+ * But we anyway can't calculate the total cost exactly, especially
+ * because we can't determine exactly when some of the workers would
+ * start executing partial plans.
+ */
+ path->total_cost = Max(highest_nonpartial_cost, path->total_cost);
+ }
+ else
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+
+ path->total_cost += subpath->total_cost;
+ if (l == list_head(subpaths)) /* first node? */
+ path->startup_cost = subpath->startup_cost;
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c80c999..c517900 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5163,7 +5165,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5174,6 +5176,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 68d74cb..1529396 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3383,8 +3383,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d88738e..4069855 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fca96eb..9f962e0 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,69 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1278,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..f8f25e6 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -494,7 +494,7 @@ RegisterLWLockTranches(void)
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 64;
+ LWLockTranchesAllocated = 128;
LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(char *));
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 291bf76..3942e8a 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index a02b154..5383509 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 4a95e16..1950192 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1c88a79..70ccdbf 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1112,10 +1112,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1123,6 +1127,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81640de..2203ab4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On 24 March 2017 at 00:38, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 23 March 2017 at 16:26, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 23 March 2017 at 05:55, Robert Haas <robertmhaas@gmail.com> wrote:
So in your example we do this:
C[0] += 20;
C[1] += 16;
C[2] += 10;
/* C[2] is smaller than C[0] or C[1] at this point, so we add the next
path to C[2] */
C[2] += 8;
/* after the previous line, C[1] is now the smallest, so add to that
entry next */
C[1] += 3;
/* now we've got C[0] = 20, C[1] = 19, C[2] = 18, so add to C[2] */
C[2] += 1;
/* final result: C[0] = 20, C[1] = 19, C[2] = 19 */Now we just take the highest entry that appears in any array, which in
this case is C[0], as the total cost.Wow. The way your final result exactly tallies with my algorithm
result is very interesting. This looks like some maths or computer
science theory that I am not aware.I am currently coding the algorithm using your method.
While I was coding this, I was considering if Path->rows also should
be calculated similar to total cost for non-partial subpath and total
cost for partial subpaths. I think for rows, we can just take
total_rows divided by workers for non-partial paths, and this
approximation should suffice. It looks odd that it be treated with the
same algorithm we chose for total cost for non-partial paths.
Attached is the patch v12, where Path->rows calculation of non-partial
paths is kept separate from the way total cost is done for non-partial
costs. rows for non-partial paths is calculated as total_rows divided
by workers as approximation. And then rows for partial paths are just
added one by one.
Meanwhile, attached is a WIP patch v10. The only change in this patch
w.r.t. the last patch (v9) is that this one has a new function defined
append_nonpartial_cost(). Just sending this to show how the algorithm
looks like; haven't yet called it.
Now append_nonpartial_cost() is used, and it is tested.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
ParallelAppend_v12.patchapplication/octet-stream; name=ParallelAppend_v12.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 2de3540..a7aad08 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3643,6 +3643,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e930731..6f51372 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -832,7 +832,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1096,6 +1096,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index b91b663..8b0ec2c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..e9e8676 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 93bda42..f8448be 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1b9005f..7b22ca5 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 474f221..44da33a 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1568,6 +1568,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index a1e1a87..6611e45 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1264,7 +1267,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1300,7 +1307,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1378,23 +1443,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1404,9 +1485,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1459,7 +1540,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1677,6 +1759,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1696,7 +1850,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a129d1e..84ab4ce 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,8 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(List *subpaths, int numpaths,
+ int parallel_workers);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1704,6 +1707,190 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ *
+ * It is the total cost units taken by all the workers to finish all the
+ * non-partial subpaths.
+ * subpaths contains non-partial paths followed by partial paths.
+ * numpaths tells the number of non-partial paths.
+ *
+ */
+static Cost
+append_nonpartial_cost(List *subpaths, int numpaths, int parallel_workers)
+{
+ Cost *costarr;
+ int arrlen;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int path_index;
+ int min_index;
+ int max_index;
+
+ if (numpaths == 0)
+ return 0;
+
+ /*
+ * Build the cost array containing costs of first n number of subpaths,
+ * where n = parallel_workers. Also, its size is kept only as long as the
+ * number of subpaths, or parallel_workers, whichever is minimum.
+ */
+ arrlen = Min(parallel_workers, numpaths);
+ costarr = (Cost *) palloc(sizeof(Cost) * arrlen);
+ path_index = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+
+ if (path_index == arrlen)
+ break;
+ costarr[path_index++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = arrlen - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+ int i;
+
+ /* Consider only the non-partial paths */
+ if (path_index++ == numpaths)
+ break;
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+ for (max_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (list_length(subpaths) == 0)
+ return;
+
+ if (!path->parallel_aware)
+ {
+ Path *subpath = (Path *) linitial(subpaths);
+
+ /*
+ * Startup cost of non-parallel-aware Append is the startup cost of
+ * first subpath.
+ */
+ path->startup_cost = subpath->startup_cost;
+
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ else /* parallel-aware */
+ {
+ double max_rows = 0;
+ double nonpartial_rows = 0;
+ int i = 0;
+
+ /* Include the non-partial paths total cost */
+ path->total_cost += append_nonpartial_cost(subpaths,
+ num_nonpartial_subpaths,
+ path->parallel_workers);
+
+ /* Calculate startup cost; also add up all the rows for later use */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (i < path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ if (i < num_nonpartial_subpaths)
+ {
+ nonpartial_rows += subpath->rows;
+
+ /* Also keep track of max rows for any given subpath */
+ max_rows = Max(max_rows, subpath->rows);
+ }
+
+ i++;
+ }
+
+ /*
+ * As an approximation, non-partial rows are calculated as total rows
+ * divided by number of workers. But if there are highly unequal number
+ * of rows across the paths, this figure might not reflect correctly.
+ * So we make a note that it also should not be less than the maximum
+ * of all the path rows.
+ */
+ nonpartial_rows /= path->parallel_workers;
+ path->rows += Max(nonpartial_rows, max_rows);
+
+ /* Calculate partial paths cost. */
+ if (list_length(subpaths) > num_nonpartial_subpaths)
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ for_each_cell(l, list_nth_cell(subpaths, num_nonpartial_subpaths))
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index c80c999..c517900 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -199,7 +199,8 @@ static CteScan *make_ctescan(List *qptlist, List *qpqual,
Index scanrelid, int ctePlanId, int cteParam);
static WorkTableScan *make_worktablescan(List *qptlist, List *qpqual,
Index scanrelid, int wtParam);
-static Append *make_append(List *appendplans, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1026,7 +1027,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5163,7 +5165,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5174,6 +5176,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 68d74cb..1529396 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3383,8 +3383,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index d88738e..4069855 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index fca96eb..9f962e0 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,69 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with workers (2, 8, 8), the Append workers should be at least
+ * 8, whereas the formula gives 2. In this case, it seems better to follow
+ * the method used for calculating parallel_workers of an unpartitioned
+ * table : log3(table_size). So we treat the UNION query as if the data
+ * belongs to a single unpartitioned table, and then derive its workers. So
+ * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan
+ * workers and b is some logarithmic base such as 2 or 3. It turns out that
+ * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1264,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1278,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..f8f25e6 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -494,7 +494,7 @@ RegisterLWLockTranches(void)
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 64;
+ LWLockTranchesAllocated = 128;
LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(char *));
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 291bf76..3942e8a 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -911,6 +911,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index a02b154..5383509 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index f856f60..c822cf2 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
@@ -1187,12 +1188,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 4a95e16..1950192 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 1c88a79..70ccdbf 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1112,10 +1112,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1123,6 +1127,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index d9a9b12..43dc72f 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -103,6 +104,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 81640de..2203ab4 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
Hi,
On 2017-03-24 21:32:57 +0530, Amit Khandekar wrote:
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c index a107545..e9e8676 100644 --- a/src/backend/executor/nodeAppend.c +++ b/src/backend/executor/nodeAppend.c @@ -59,9 +59,47 @@#include "executor/execdebug.h" #include "executor/nodeAppend.h" +#include "miscadmin.h" +#include "optimizer/cost.h" +#include "storage/spin.h" + +/* + * Shared state for Parallel Append. + * + * Each backend participating in a Parallel Append has its own + * descriptor in backend-private memory, and those objects all contain + * a pointer to this structure. + */ +typedef struct ParallelAppendDescData +{ + LWLock pa_lock; /* mutual exclusion to choose next subplan */ + int pa_first_plan; /* plan to choose while wrapping around plans */ + int pa_next_plan; /* next plan to choose by any worker */ + + /* + * pa_finished : workers currently executing the subplan. A worker which + * finishes a subplan should set pa_finished to true, so that no new + * worker picks this subplan. For non-partial subplan, a worker which picks + * up that subplan should immediately set to true, so as to make sure + * there are no more than 1 worker assigned to this subplan. + */ + bool pa_finished[FLEXIBLE_ARRAY_MEMBER]; +} ParallelAppendDescData;
+typedef ParallelAppendDescData *ParallelAppendDesc;
Pointer hiding typedefs make this Andres sad.
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node) if (subnode->chgParam == NULL) ExecReScan(subnode); } + + if (padesc) + { + padesc->pa_first_plan = padesc->pa_next_plan = 0; + memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans); + } +
Is it actually guaranteed that none of the parallel workers are doing
something at that point?
+/* ---------------------------------------------------------------- + * exec_append_parallel_next + * + * Determine the next subplan that should be executed. Each worker uses a + * shared next_subplan counter index to start looking for unfinished plan, + * executes the subplan, then shifts ahead this counter to the next + * subplan, so that other workers know which next plan to choose. This + * way, workers choose the subplans in round robin order, and thus they + * get evenly distributed among the subplans. + * + * Returns false if and only if all subplans are already finished + * processing. + * ---------------------------------------------------------------- + */ +static bool +exec_append_parallel_next(AppendState *state) +{ + ParallelAppendDesc padesc = state->as_padesc; + int whichplan; + int initial_plan; + int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan; + bool found; + + Assert(padesc != NULL); + + /* Backward scan is not supported by parallel-aware plans */ + Assert(ScanDirectionIsForward(state->ps.state->es_direction)); + + /* The parallel leader chooses its next subplan differently */ + if (!IsParallelWorker()) + return exec_append_leader_next(state);
It's a bit weird that the leader's case does is so separate, and does
it's own lock acquisition.
+ found = false; + for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;) + { + /* + * Ignore plans that are already done processing. These also include + * non-partial subplans which have already been taken by a worker. + */ + if (!padesc->pa_finished[whichplan]) + { + found = true; + break; + } + + /* + * Note: There is a chance that just after the child plan node is + * chosen above, some other worker finishes this node and sets + * pa_finished to true. In that case, this worker will go ahead and + * call ExecProcNode(child_node), which will return NULL tuple since it + * is already finished, and then once again this worker will try to + * choose next subplan; but this is ok : it's just an extra + * "choose_next_subplan" operation. + */
IIRC not all node types are safe against being executed again when
they've previously returned NULL. That's why e.g. nodeMaterial.c
contains the following blurb:
/*
* If necessary, try to fetch another row from the subplan.
*
* Note: the eof_underlying state variable exists to short-circuit further
* subplan calls. It's not optional, unfortunately, because some plan
* node types are not robust about being called again when they've already
* returned NULL.
*/
+ else if (IsA(subpath, MergeAppendPath)) + { + MergeAppendPath *mpath = (MergeAppendPath *) subpath; + + /* + * If at all MergeAppend is partial, all its child plans have to be + * partial : we don't currently support a mix of partial and + * non-partial MergeAppend subpaths. + */
Why is that?
+int +get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths) +{ + ListCell *lc; + double log2w; + int num_workers; + int max_per_plan_workers; + + /* + * log2(number_of_subpaths)+1 formula seems to give an appropriate number of + * workers for Append path either having high number of children (> 100) or + * having all non-partial subpaths or subpaths with 1-2 parallel_workers. + * Whereas, if the subpaths->parallel_workers is high, this formula is not + * suitable, because it does not take into account per-subpath workers. + * For e.g., with workers (2, 8, 8),
That's the per-subplan workers for three subplans? That's not
necessarily clear.
the Append workers should be at least + * 8, whereas the formula gives 2. In this case, it seems better to follow + * the method used for calculating parallel_workers of an unpartitioned + * table : log3(table_size). So we treat the UNION query as if the data
Which "UNION query"?
+ * belongs to a single unpartitioned table, and then derive its workers. So + * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan + * workers and b is some logarithmic base such as 2 or 3. It turns out that + * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we + * just use the maximum of workers formula. But this formula gives too few + * workers when all paths have single worker (meaning they are non-partial) + * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3 + * workers, whereas this method allocates only 1. + * So we use whichever method that gives higher number of workers. + */ + + /* Get log2(num_subpaths) */ + log2w = fls(list_length(partial_subpaths) + + list_length(nonpartial_subpaths)); + + /* Avoid further calculations if we already crossed max workers limit */ + if (max_parallel_workers_per_gather <= log2w + 1) + return max_parallel_workers_per_gather; + + + /* + * Get the parallel_workers value of the partial subpath having the highest + * parallel_workers. + */ + max_per_plan_workers = 1; + foreach(lc, partial_subpaths) + { + Path *subpath = lfirst(lc); + max_per_plan_workers = Max(max_per_plan_workers, + subpath->parallel_workers); + } + + /* Choose the higher of the results of the two formulae */ + num_workers = rint(Max(log2w, max_per_plan_workers) + 1); + + /* In no case use more than max_parallel_workers_per_gather workers. */ + num_workers = Min(num_workers, max_parallel_workers_per_gather); + + return num_workers; +}
Hm. I'm not really convinced by the logic here. Wouldn't it be better
to try to compute the minimum total cost across all workers for
1..#max_workers for the plans in an iterative manner? I.e. try to map
each of the subplans to 1 (if non-partial) or N workers (partial) using
some fitting algorith (e.g. always choosing the worker(s) that currently
have the least work assigned). I think the current algorithm doesn't
lead to useful #workers for e.g. cases with a lot of non-partial,
high-startup plans - imo a quite reasonable scenario.
I'm afraid this is too late for v10 - do you agree?
- Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Apr 3, 2017 at 4:17 PM, Andres Freund <andres@anarazel.de> wrote:
Hm. I'm not really convinced by the logic here. Wouldn't it be better
to try to compute the minimum total cost across all workers for
1..#max_workers for the plans in an iterative manner? I.e. try to map
each of the subplans to 1 (if non-partial) or N workers (partial) using
some fitting algorith (e.g. always choosing the worker(s) that currently
have the least work assigned). I think the current algorithm doesn't
lead to useful #workers for e.g. cases with a lot of non-partial,
high-startup plans - imo a quite reasonable scenario.
Well, that'd be totally unlike what we do in any other case. We only
generate a Parallel Seq Scan plan for a given table with one # of
workers, and we cost it based on that. We have no way to re-cost it
if we changed our mind later about how many workers to use.
Eventually, we should probably have something like what you're
describing here, but in general, not just for this specific case. One
problem, of course, is to avoid having a larger number of workers
always look better than a smaller number, which with the current
costing model would probably happen a lot.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-04-03 22:13:18 -0400, Robert Haas wrote:
On Mon, Apr 3, 2017 at 4:17 PM, Andres Freund <andres@anarazel.de> wrote:
Hm. I'm not really convinced by the logic here. Wouldn't it be better
to try to compute the minimum total cost across all workers for
1..#max_workers for the plans in an iterative manner? I.e. try to map
each of the subplans to 1 (if non-partial) or N workers (partial) using
some fitting algorith (e.g. always choosing the worker(s) that currently
have the least work assigned). I think the current algorithm doesn't
lead to useful #workers for e.g. cases with a lot of non-partial,
high-startup plans - imo a quite reasonable scenario.Well, that'd be totally unlike what we do in any other case. We only
generate a Parallel Seq Scan plan for a given table with one # of
workers, and we cost it based on that. We have no way to re-cost it
if we changed our mind later about how many workers to use.
Eventually, we should probably have something like what you're
describing here, but in general, not just for this specific case. One
problem, of course, is to avoid having a larger number of workers
always look better than a smaller number, which with the current
costing model would probably happen a lot.
I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work, and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Thanks Andres for your review comments. Will get back with the other
comments, but meanwhile some queries about the below particular
comment ...
On 4 April 2017 at 10:17, Andres Freund <andres@anarazel.de> wrote:
On 2017-04-03 22:13:18 -0400, Robert Haas wrote:
On Mon, Apr 3, 2017 at 4:17 PM, Andres Freund <andres@anarazel.de> wrote:
Hm. I'm not really convinced by the logic here. Wouldn't it be better
to try to compute the minimum total cost across all workers for
1..#max_workers for the plans in an iterative manner? I.e. try to map
each of the subplans to 1 (if non-partial) or N workers (partial) using
some fitting algorith (e.g. always choosing the worker(s) that currently
have the least work assigned). I think the current algorithm doesn't
lead to useful #workers for e.g. cases with a lot of non-partial,
high-startup plans - imo a quite reasonable scenario.
I think I might have not understood this part exactly. Are you saying
we need to consider per-subplan parallel_workers to calculate total
number of workers for Append ? I also didn't get about non-partial
subplans. Can you please explain how many workers you think should be
expected with , say , 7 subplans out of which 3 are non-partial
subplans ?
Well, that'd be totally unlike what we do in any other case. We only
generate a Parallel Seq Scan plan for a given table with one # of
workers, and we cost it based on that. We have no way to re-cost it
if we changed our mind later about how many workers to use.
Eventually, we should probably have something like what you're
describing here, but in general, not just for this specific case. One
problem, of course, is to avoid having a larger number of workers
always look better than a smaller number, which with the current
costing model would probably happen a lot.I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work, and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.Greetings,
Andres Freund
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 4 April 2017 at 01:47, Andres Freund <andres@anarazel.de> wrote:
+typedef struct ParallelAppendDescData +{ + LWLock pa_lock; /* mutual exclusion to choose next subplan */ + int pa_first_plan; /* plan to choose while wrapping around plans */ + int pa_next_plan; /* next plan to choose by any worker */ + + /* + * pa_finished : workers currently executing the subplan. A worker which + * finishes a subplan should set pa_finished to true, so that no new + * worker picks this subplan. For non-partial subplan, a worker which picks + * up that subplan should immediately set to true, so as to make sure + * there are no more than 1 worker assigned to this subplan. + */ + bool pa_finished[FLEXIBLE_ARRAY_MEMBER]; +} ParallelAppendDescData;+typedef ParallelAppendDescData *ParallelAppendDesc;
Pointer hiding typedefs make this Andres sad.
Yeah .. was trying to be consistent with other parts of code where we
have typedefs for both structure and a pointer to that structure.
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node) if (subnode->chgParam == NULL) ExecReScan(subnode); } + + if (padesc) + { + padesc->pa_first_plan = padesc->pa_next_plan = 0; + memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans); + } +Is it actually guaranteed that none of the parallel workers are doing
something at that point?
ExecReScanAppend() would be called by ExecReScanGather().
ExecReScanGather() shuts down all the parallel workers before calling
its child node (i.e. ExecReScanAppend).
+static bool +exec_append_parallel_next(AppendState *state) +{ + ParallelAppendDesc padesc = state->as_padesc; + int whichplan; + int initial_plan; + int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan; + bool found; + + Assert(padesc != NULL); + + /* Backward scan is not supported by parallel-aware plans */ + Assert(ScanDirectionIsForward(state->ps.state->es_direction)); + + /* The parallel leader chooses its next subplan differently */ + if (!IsParallelWorker()) + return exec_append_leader_next(state);It's a bit weird that the leader's case does is so separate, and does
it's own lock acquisition.
Since we wanted to prevent it from taking the most expensive
non-partial plans first , thought it would be better to keep its logic
simple and separate, so could not merge it in the main logic code.
+ found = false; + for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;) + { + /* + * Ignore plans that are already done processing. These also include + * non-partial subplans which have already been taken by a worker. + */ + if (!padesc->pa_finished[whichplan]) + { + found = true; + break; + } + + /* + * Note: There is a chance that just after the child plan node is + * chosen above, some other worker finishes this node and sets + * pa_finished to true. In that case, this worker will go ahead and + * call ExecProcNode(child_node), which will return NULL tuple since it + * is already finished, and then once again this worker will try to + * choose next subplan; but this is ok : it's just an extra + * "choose_next_subplan" operation. + */IIRC not all node types are safe against being executed again when
they've previously returned NULL. That's why e.g. nodeMaterial.c
contains the following blurb:
/*
* If necessary, try to fetch another row from the subplan.
*
* Note: the eof_underlying state variable exists to short-circuit further
* subplan calls. It's not optional, unfortunately, because some plan
* node types are not robust about being called again when they've already
* returned NULL.
*/
This scenario is different from the parallel append scenario described
by my comment. A worker sets pa_finished to true only when it itself
gets a NULL tuple for a given subplan. So in
exec_append_parallel_next(), suppose a worker W1 finds a subplan with
pa_finished=false. So it chooses it. Now a different worker W2 sets
this subplan's pa_finished=true because W2 has got a NULL tuple. But
W1 hasn't yet got a NULL tuple. If it had got a NULL tuple earlier, it
would have itself set pa_finished to true, and then it would have
never again chosen this subplan. So effectively, a worker would never
execute the same subplan once that subplan returns NULL.
+ else if (IsA(subpath, MergeAppendPath)) + { + MergeAppendPath *mpath = (MergeAppendPath *) subpath; + + /* + * If at all MergeAppend is partial, all its child plans have to be + * partial : we don't currently support a mix of partial and + * non-partial MergeAppend subpaths. + */Why is that?
The mix of partial and non-partial subplans is being implemented only
for Append plan. In the future if and when we extend this support for
MergeAppend, then we would need to change this. Till then, we can
assume that if MergeAppend is partial, all it child plans have to be
partial otherwise there woudn't have been a partial MergeAppendPath.
BTW MergeAppendPath currently is itself never partial. So in the
comment it is mentioned "If at all MergeAppend is partial".
+int +get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths) +{ + ListCell *lc; + double log2w; + int num_workers; + int max_per_plan_workers; + + /* + * log2(number_of_subpaths)+1 formula seems to give an appropriate number of + * workers for Append path either having high number of children (> 100) or + * having all non-partial subpaths or subpaths with 1-2 parallel_workers. + * Whereas, if the subpaths->parallel_workers is high, this formula is not + * suitable, because it does not take into account per-subpath workers. + * For e.g., with workers (2, 8, 8),That's the per-subplan workers for three subplans? That's not
necessarily clear.
Right. Corrected it to : "3 subplans having per-subplan workers such
as (2, 8, 8)"
the Append workers should be at least + * 8, whereas the formula gives 2. In this case, it seems better to follow + * the method used for calculating parallel_workers of an unpartitioned + * table : log3(table_size). So we treat the UNION query as if the dataWhich "UNION query"?
Changed it to "partitioned table". The idea is : treat all the data of
a partitioned table as if it belonged to a single non-partitioned
table, and then calculate the workers for such a table. It may not
exactly apply for UNION query because that can involve different
tables and with joins too. So replaced UNION query to partitioned
table.
+ * belongs to a single unpartitioned table, and then derive its workers. So + * it will be : logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan + * workers and b is some logarithmic base such as 2 or 3. It turns out that + * this evaluates to a value just a bit greater than max(w1,w2, w3). So, we + * just use the maximum of workers formula. But this formula gives too few + * workers when all paths have single worker (meaning they are non-partial) + * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3 + * workers, whereas this method allocates only 1. + * So we use whichever method that gives higher number of workers. + */ + + /* Get log2(num_subpaths) */ + log2w = fls(list_length(partial_subpaths) + + list_length(nonpartial_subpaths)); + + /* Avoid further calculations if we already crossed max workers limit */ + if (max_parallel_workers_per_gather <= log2w + 1) + return max_parallel_workers_per_gather; + + + /* + * Get the parallel_workers value of the partial subpath having the highest + * parallel_workers. + */ + max_per_plan_workers = 1; + foreach(lc, partial_subpaths) + { + Path *subpath = lfirst(lc); + max_per_plan_workers = Max(max_per_plan_workers, + subpath->parallel_workers); + } + + /* Choose the higher of the results of the two formulae */ + num_workers = rint(Max(log2w, max_per_plan_workers) + 1); + + /* In no case use more than max_parallel_workers_per_gather workers. */ + num_workers = Min(num_workers, max_parallel_workers_per_gather); + + return num_workers; +}Hm. I'm not really convinced by the logic here. Wouldn't it be better
to try to compute the minimum total cost across all workers for
1..#max_workers for the plans in an iterative manner? I.e. try to map
each of the subplans to 1 (if non-partial) or N workers (partial) using
some fitting algorith (e.g. always choosing the worker(s) that currently
have the least work assigned). I think the current algorithm doesn't
lead to useful #workers for e.g. cases with a lot of non-partial,
high-startup plans - imo a quite reasonable scenario.
Have responded in a separate reply.
I'm afraid this is too late for v10 - do you agree?
I am not exactly sure; may be it depends upon how much more review
comments would follow this week. I anticipate there would not be any
high level/design-level changes now.
Attached is an updated patch v13 that has some comments changed as per
your review, and also rebased on latest master.
Attachments:
ParallelAppend_v13.patchapplication/octet-stream; name=ParallelAppend_v13.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index ac339fb..59d24c0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3643,6 +3643,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 9856968..f4c78e4 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -843,7 +843,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1107,6 +1107,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 469a32c..cc8422c 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -215,6 +216,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -279,6 +284,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -782,6 +791,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index a107545..e9e8676 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 61bc502..e939dda 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -239,6 +239,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index f09aa24..d5e3ca7 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 83fb39f..b60bc16 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -372,6 +372,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 766f2d8..cb246f2 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1575,6 +1575,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b93b4fc..67f7c89 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1280,7 +1283,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1316,7 +1323,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1394,23 +1459,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1420,9 +1501,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1475,7 +1556,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1693,6 +1775,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1712,7 +1866,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index ed07e2f..4179145 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,8 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(List *subpaths, int numpaths,
+ int parallel_workers);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1741,6 +1744,190 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ *
+ * It is the total cost units taken by all the workers to finish all the
+ * non-partial subpaths.
+ * subpaths contains non-partial paths followed by partial paths.
+ * numpaths tells the number of non-partial paths.
+ *
+ */
+static Cost
+append_nonpartial_cost(List *subpaths, int numpaths, int parallel_workers)
+{
+ Cost *costarr;
+ int arrlen;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int path_index;
+ int min_index;
+ int max_index;
+
+ if (numpaths == 0)
+ return 0;
+
+ /*
+ * Build the cost array containing costs of first n number of subpaths,
+ * where n = parallel_workers. Also, its size is kept only as long as the
+ * number of subpaths, or parallel_workers, whichever is minimum.
+ */
+ arrlen = Min(parallel_workers, numpaths);
+ costarr = (Cost *) palloc(sizeof(Cost) * arrlen);
+ path_index = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+
+ if (path_index == arrlen)
+ break;
+ costarr[path_index++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = arrlen - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+ int i;
+
+ /* Consider only the non-partial paths */
+ if (path_index++ == numpaths)
+ break;
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+ for (max_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (list_length(subpaths) == 0)
+ return;
+
+ if (!path->parallel_aware)
+ {
+ Path *subpath = (Path *) linitial(subpaths);
+
+ /*
+ * Startup cost of non-parallel-aware Append is the startup cost of
+ * first subpath.
+ */
+ path->startup_cost = subpath->startup_cost;
+
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ else /* parallel-aware */
+ {
+ double max_rows = 0;
+ double nonpartial_rows = 0;
+ int i = 0;
+
+ /* Include the non-partial paths total cost */
+ path->total_cost += append_nonpartial_cost(subpaths,
+ num_nonpartial_subpaths,
+ path->parallel_workers);
+
+ /* Calculate startup cost; also add up all the rows for later use */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (i < path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ if (i < num_nonpartial_subpaths)
+ {
+ nonpartial_rows += subpath->rows;
+
+ /* Also keep track of max rows for any given subpath */
+ max_rows = Max(max_rows, subpath->rows);
+ }
+
+ i++;
+ }
+
+ /*
+ * As an approximation, non-partial rows are calculated as total rows
+ * divided by number of workers. But if there are highly unequal number
+ * of rows across the paths, this figure might not reflect correctly.
+ * So we make a note that it also should not be less than the maximum
+ * of all the path rows.
+ */
+ nonpartial_rows /= path->parallel_workers;
+ path->rows += Max(nonpartial_rows, max_rows);
+
+ /* Calculate partial paths cost. */
+ if (list_length(subpaths) > num_nonpartial_subpaths)
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ for_each_cell(l, list_nth_cell(subpaths, num_nonpartial_subpaths))
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6a0c67b..6e39fc1 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index b121f40..b1f9dc6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -203,7 +203,8 @@ 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, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1038,7 +1039,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5245,7 +5247,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5256,6 +5258,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 17cd683..85a1110 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3587,8 +3587,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index b5cb4de..8f18841 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -566,8 +566,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -678,7 +678,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 8536212..a589c22 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,70 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with 3 subplans having per-subplan workers such as (2, 8, 8),
+ * the Append workers should be at least 8, whereas the formula gives 2. In
+ * this case, it seems better to follow the method used for calculating
+ * parallel_workers of an unpartitioned table : log3(table_size). So we
+ * treat a partitioned table as if the data belongs to a single
+ * unpartitioned table, and then derive its workers. So it will be :
+ * logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan workers and
+ * b is some logarithmic base such as 2 or 3. It turns out that this
+ * evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1265,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,44 +1279,51 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered
* unsorted */
pathnode->partitioned_rels = partitioned_rels;
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 3e13394..f8f25e6 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -494,7 +494,7 @@ RegisterLWLockTranches(void)
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 64;
+ LWLockTranchesAllocated = 128;
LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(char *));
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 8b5f064..580d8e0 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -912,6 +912,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 8a93bdc..141bd92 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -288,6 +288,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 6fb4662..e76027f 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index fa99244..58fc0ed 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/queryenvironment.h"
#include "utils/reltrigger.h"
@@ -944,12 +945,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 90e84bc..8350220 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -248,6 +248,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index a2dd26f..f481532 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -235,6 +235,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index fc53eb1..1e64c1c 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1151,10 +1151,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1162,6 +1166,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6909359..d71463d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -105,6 +106,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 2e712c6..04f1f32 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 0cd45bb..802a380 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 6163ed8..49d232f 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 038a62e..6ffe23d 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index d43b75c..2270c53 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index 9311a77..0623319 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Tue, Apr 4, 2017 at 12:47 AM, Andres Freund <andres@anarazel.de> wrote:
I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work, and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.
Sure, that's a problem, but I think it's still absolutely necessary to
ramp up the maximum "effort" (in terms of number of workers)
logarithmically. If you just do it by costing, the winning number of
workers will always be the largest number that we think we'll be able
to put to use - e.g. with 100 branches of relatively equal cost we'll
pick 100 workers. That's not remotely sane.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Apr 3, 2017 at 4:17 PM, Andres Freund <andres@anarazel.de> wrote:
I'm afraid this is too late for v10 - do you agree?
Yeah, I think so. The benefit of this will be a lot more once we get
partitionwise join and partitionwise aggregate working, but that
probably won't happen for this release, or at best in limited cases.
And while we might not agree on exactly what work this patch still
needs, I think it does still need some work. I've moved this to the
next CommitFest.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-04-04 08:01:32 -0400, Robert Haas wrote:
On Tue, Apr 4, 2017 at 12:47 AM, Andres Freund <andres@anarazel.de> wrote:
I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work, and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.Sure, that's a problem, but I think it's still absolutely necessary to
ramp up the maximum "effort" (in terms of number of workers)
logarithmically. If you just do it by costing, the winning number of
workers will always be the largest number that we think we'll be able
to put to use - e.g. with 100 branches of relatively equal cost we'll
pick 100 workers. That's not remotely sane.
I'm quite unconvinced that just throwing a log() in there is the best
way to combat that. Modeling the issue of starting more workers through
tuple transfer, locking, startup overhead costing seems a better to me.
If the goal is to compute the results of the query as fast as possible,
and to not use more than max_parallel_per_XXX, and it's actually
beneficial to use more workers, then we should. Because otherwise you
really can't use the resources available.
- Andres
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Apr 5, 2017 at 1:43 AM, Andres Freund <andres@anarazel.de> wrote:
On 2017-04-04 08:01:32 -0400, Robert Haas wrote:
On Tue, Apr 4, 2017 at 12:47 AM, Andres Freund <andres@anarazel.de>
wrote:
I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work,and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.Sure, that's a problem, but I think it's still absolutely necessary to
ramp up the maximum "effort" (in terms of number of workers)
logarithmically. If you just do it by costing, the winning number of
workers will always be the largest number that we think we'll be able
to put to use - e.g. with 100 branches of relatively equal cost we'll
pick 100 workers. That's not remotely sane.I'm quite unconvinced that just throwing a log() in there is the best
way to combat that. Modeling the issue of starting more workers through
tuple transfer, locking, startup overhead costing seems a better to me.If the goal is to compute the results of the query as fast as possible,
and to not use more than max_parallel_per_XXX, and it's actually
beneficial to use more workers, then we should. Because otherwise you
really can't use the resources available.
+1. I had expressed similar opinion earlier, but yours is better
articulated. Thanks.
--
Best Wishes,
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
On 5 April 2017 at 01:43, Andres Freund <andres@anarazel.de> wrote:
On 2017-04-04 08:01:32 -0400, Robert Haas wrote:
On Tue, Apr 4, 2017 at 12:47 AM, Andres Freund <andres@anarazel.de> wrote:
I don't think the parallel seqscan is comparable in complexity with the
parallel append case. Each worker there does the same kind of work, and
if one of them is behind, it'll just do less. But correct sizing will
be more important with parallel-append, because with non-partial
subplans the work is absolutely *not* uniform.Sure, that's a problem, but I think it's still absolutely necessary to
ramp up the maximum "effort" (in terms of number of workers)
logarithmically. If you just do it by costing, the winning number of
workers will always be the largest number that we think we'll be able
to put to use - e.g. with 100 branches of relatively equal cost we'll
pick 100 workers. That's not remotely sane.I'm quite unconvinced that just throwing a log() in there is the best
way to combat that. Modeling the issue of starting more workers through
tuple transfer, locking, startup overhead costing seems a better to me.If the goal is to compute the results of the query as fast as possible,
and to not use more than max_parallel_per_XXX, and it's actually
beneficial to use more workers, then we should. Because otherwise you
really can't use the resources available.- Andres
This is what the earlier versions of my patch had done : just add up
per-subplan parallel_workers (1 for non-partial subplan and
subpath->parallel_workers for partial subplans) and set this total as
the Append parallel_workers.
Robert had a valid point that this would be inconsistent with the
worker count that we would come up with if it were a single table with
a cost as big as the total cost of all Append subplans. We were
discussing rather about partitioned table versus if it were
unpartitioned, but I think the same argument goes for a union query
with non-partial plans : if we want to clamp down the number of
workers for a single table for a good reason, we should then also
follow that policy and prevent assigning too many workers even for an
Append.
Now I am not sure of the reason why for a single table parallel scan,
we increase number of workers logarithmically; but I think there might
have been an observation that after certain number of workers, adding
up more workers does not make significant difference, but this is just
my guess.
If we try to calculate workers based on each of the subplan costs
rather than just the number of workers, still I think the total worker
count should be a *log* of the total cost, so as to be consistent with
what we did for other scans. Now log(total_cost) does not increase
significantly with cost. For cost of 1000 units, the log3(cost) will
be 6, and for cost of 10,000 units, it is 8, i.e. just 2 more workers.
So I think since its a logarithmic value, it would be might as well
better to just drop the cost factor, and consider only number of
workers.
But again, in the future if we drop the method of log(), then the
above is not valid. But I think till then we should follow some common
strategy we have been following.
BTW all of the above points apply only for non-partial plans. For
partial plans, what we have done in the patch is : Take the highest of
the per-subplan parallel_workers, and make sure that Append workers is
at least as high as this value.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Apr 4, 2017 at 4:13 PM, Andres Freund <andres@anarazel.de> wrote:
I'm quite unconvinced that just throwing a log() in there is the best
way to combat that. Modeling the issue of starting more workers through
tuple transfer, locking, startup overhead costing seems a better to me.
Knock yourself out. There's no doubt that the way the number of
parallel workers is computed is pretty stupid right now, and it
obviously needs to get a lot smarter before we can consider doing
things like throwing 40 workers at a query. If you throw 2 or 4
workers at a query and it turns out that it doesn't help much, that's
sad, but if you throw 40 workers at a query and it turns out that it
doesn't help much, or even regresses, that's a lot sadder. The
existing system does try to model startup and tuple transfer overhead
during costing, but only as a way of comparing parallel plans to each
other or to non-parallel plans, not to work out the right number of
workers. It also does not model contention, which it absolutely needs
to do. I was kind of hoping that once the first version of parallel
query was committed, other developers who care about the query planner
would be motivated to improve some of this stuff, but so far that
hasn't really happened. This release adds a decent number of new
execution capabilities, and there is a lot more work to be done there,
but without some serious work on the planner end of things I fear
we're never going to be able to get more than ~4x speedup out of
parallel query, because we're just too dumb to know how many workers
we really ought to be using.
That having been said, I completely and emphatically disagree that
this patch ought to be required to be an order-of-magnitude smarter
than the existing logic in order to get committed. There are four
main things that this patch can hope to accomplish:
1. If we've got an Append node with children that have a non-zero
startup cost, it is currently pretty much guaranteed that every worker
will pay the startup cost for every child. With Parallel Append, we
can spread out the workers across the plans, and once a plan has been
finished by however many workers it got, other workers can ignore it,
which means that its startup cost need not be paid by those workers.
This case will arise a lot more frequently once we have partition-wise
join.
2. When the Append node's children are partial plans, spreading out
the workers reduces contention for whatever locks those workers use to
coordinate access to shared data.
3. If the Append node represents a scan of a partitioned table, and
the partitions are on different tablespaces (or there's just enough
I/O bandwidth available in a single tablespace to read more than one
of them at once without slowing things down), then spreading out the
work gives us I/O parallelism. This is an area where some
experimentation and benchmarking is needed, because there is a
possibility of regressions if we run several sequential scans on the
same spindle in parallel instead of consecutively. We might need to
add some logic to try to avoid this, but it's not clear how that logic
should work.
4. If the Append node is derived from a UNION ALL query, we can run
different branches in different processes even if the plans are not
themselves able to be parallelized. This was proposed by Stephen
among others as an "easy" case for parallelism, which was maybe a tad
optimistic, but it's sad that we're going to release v10 without
having done anything about it.
All of those things (except possibly #3) are wins over the status quo
even if the way we choose the number of workers is still pretty dumb.
It shouldn't get away with being dumber than what we've already got,
but it shouldn't be radically smarter - or even just radically
different because, if it is, then the results you get when you query a
partitioned table will be very different from what you get when you
query an partitioned table, which is not sensible. I very much agree
that doing something smarter than log-scaling on the number of workers
is an a good project for somebody to do, but it's not *this* project.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2017-04-05 14:52:38 +0530, Amit Khandekar wrote:
This is what the earlier versions of my patch had done : just add up
per-subplan parallel_workers (1 for non-partial subplan and
subpath->parallel_workers for partial subplans) and set this total as
the Append parallel_workers.
I don't think that's great, consider e.g. the case that you have one
very expensive query, and a bunch of cheaper ones. Most of those workers
wouldn't do much while waiting for the the expensive query. What I'm
basically thinking we should do is something like the following
pythonesque pseudocode:
best_nonpartial_cost = -1
best_nonpartial_nworkers = -1
for numworkers in 1...#max workers:
worker_work = [0 for x in range(0, numworkers)]
nonpartial_cost += startup_cost * numworkers
# distribute all nonpartial tasks over workers. Assign tasks to the
# worker with the least amount of work already performed.
for task in all_nonpartial_subqueries:
least_busy_worker = worker_work.smallest()
least_busy_worker += task.total_nonpartial_cost
# the nonpartial cost here is the largest amount any single worker
# has to perform.
nonpartial_cost += worker_work.largest()
total_partial_cost = 0
for task in all_partial_subqueries:
total_partial_cost += task.total_nonpartial_cost
# Compute resources needed by partial tasks. First compute how much
# cost we can distribute to workers that take shorter than the
# "busiest" worker doing non-partial tasks.
remaining_avail_work = 0
for i in range(0, numworkers):
remaining_avail_work += worker_work.largest() - worker_work[i]
# Equally divide up remaining work over all workers
if remaining_avail_work < total_partial_cost:
nonpartial_cost += (worker_work.largest - remaining_avail_work) / numworkers
# check if this is the best number of workers
if best_nonpartial_cost == -1 or best_nonpartial_cost > nonpartial_cost:
best_nonpartial_cost = worker_work.largest
best_nonpartial_nworkers = nworkers
Does that make sense?
BTW all of the above points apply only for non-partial plans.
Indeed. But I think that's going to be a pretty common type of plan,
especially if we get partitionwise joins.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 6 April 2017 at 07:33, Andres Freund <andres@anarazel.de> wrote:
On 2017-04-05 14:52:38 +0530, Amit Khandekar wrote:
This is what the earlier versions of my patch had done : just add up
per-subplan parallel_workers (1 for non-partial subplan and
subpath->parallel_workers for partial subplans) and set this total as
the Append parallel_workers.I don't think that's great, consider e.g. the case that you have one
very expensive query, and a bunch of cheaper ones. Most of those workers
wouldn't do much while waiting for the the expensive query. What I'm
basically thinking we should do is something like the following
pythonesque pseudocode:best_nonpartial_cost = -1
best_nonpartial_nworkers = -1for numworkers in 1...#max workers:
worker_work = [0 for x in range(0, numworkers)]nonpartial_cost += startup_cost * numworkers
# distribute all nonpartial tasks over workers. Assign tasks to the
# worker with the least amount of work already performed.
for task in all_nonpartial_subqueries:
least_busy_worker = worker_work.smallest()
least_busy_worker += task.total_nonpartial_cost# the nonpartial cost here is the largest amount any single worker
# has to perform.
nonpartial_cost += worker_work.largest()total_partial_cost = 0
for task in all_partial_subqueries:
total_partial_cost += task.total_nonpartial_cost# Compute resources needed by partial tasks. First compute how much
# cost we can distribute to workers that take shorter than the
# "busiest" worker doing non-partial tasks.
remaining_avail_work = 0
for i in range(0, numworkers):
remaining_avail_work += worker_work.largest() - worker_work[i]# Equally divide up remaining work over all workers
if remaining_avail_work < total_partial_cost:
nonpartial_cost += (worker_work.largest - remaining_avail_work) / numworkers# check if this is the best number of workers
if best_nonpartial_cost == -1 or best_nonpartial_cost > nonpartial_cost:
best_nonpartial_cost = worker_work.largest
best_nonpartial_nworkers = nworkersDoes that make sense?
Yeah, I gather what you are trying to achieve is : allocate number of
workers such that the total cost does not exceed the cost of the first
non-partial plan (i.e. the costliest one, because the plans are sorted
by descending cost).
So for non-partial costs such as (20, 10, 5, 2) allocate only 2
workers because the 2nd worker will execute (10, 5, 2) while 1st
worker executes (20).
But for costs such as (4, 4, 4, .... 20 times), the logic would give
us 20 workers because we want to finish the Append in 4 time units;
and this what we want to avoid when we go with
don't-allocate-too-many-workers approach.
BTW all of the above points apply only for non-partial plans.
Indeed. But I think that's going to be a pretty common type of plan,
Yes it is.
especially if we get partitionwise joins.
About that I am not sure, because we already have support for parallel
joins, so wouldn't the join subpaths corresponding to all of the
partitions be partial paths ? I may be wrong about that.
But if the subplans are foreign scans, then yes all would be
non-partial plans. This may provoke off-topic discussion, but here
instead of assigning so many workers to all these foreign plans and
all those workers waiting for the results, a single asynchronous
execution node (which is still in the making) would be desirable
because it would do the job of all these workers.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
On 2017-04-07 11:44:39 +0530, Amit Khandekar wrote:
On 6 April 2017 at 07:33, Andres Freund <andres@anarazel.de> wrote:
On 2017-04-05 14:52:38 +0530, Amit Khandekar wrote:
This is what the earlier versions of my patch had done : just add up
per-subplan parallel_workers (1 for non-partial subplan and
subpath->parallel_workers for partial subplans) and set this total as
the Append parallel_workers.I don't think that's great, consider e.g. the case that you have one
very expensive query, and a bunch of cheaper ones. Most of those workers
wouldn't do much while waiting for the the expensive query. What I'm
basically thinking we should do is something like the following
pythonesque pseudocode:best_nonpartial_cost = -1
best_nonpartial_nworkers = -1for numworkers in 1...#max workers:
worker_work = [0 for x in range(0, numworkers)]nonpartial_cost += startup_cost * numworkers
# distribute all nonpartial tasks over workers. Assign tasks to the
# worker with the least amount of work already performed.
for task in all_nonpartial_subqueries:
least_busy_worker = worker_work.smallest()
least_busy_worker += task.total_nonpartial_cost# the nonpartial cost here is the largest amount any single worker
# has to perform.
nonpartial_cost += worker_work.largest()total_partial_cost = 0
for task in all_partial_subqueries:
total_partial_cost += task.total_nonpartial_cost# Compute resources needed by partial tasks. First compute how much
# cost we can distribute to workers that take shorter than the
# "busiest" worker doing non-partial tasks.
remaining_avail_work = 0
for i in range(0, numworkers):
remaining_avail_work += worker_work.largest() - worker_work[i]# Equally divide up remaining work over all workers
if remaining_avail_work < total_partial_cost:
nonpartial_cost += (worker_work.largest - remaining_avail_work) / numworkers# check if this is the best number of workers
if best_nonpartial_cost == -1 or best_nonpartial_cost > nonpartial_cost:
best_nonpartial_cost = worker_work.largest
best_nonpartial_nworkers = nworkersDoes that make sense?
Yeah, I gather what you are trying to achieve is : allocate number of
workers such that the total cost does not exceed the cost of the first
non-partial plan (i.e. the costliest one, because the plans are sorted
by descending cost).So for non-partial costs such as (20, 10, 5, 2) allocate only 2
workers because the 2nd worker will execute (10, 5, 2) while 1st
worker executes (20).But for costs such as (4, 4, 4, .... 20 times), the logic would give
us 20 workers because we want to finish the Append in 4 time units;
and this what we want to avoid when we go with
don't-allocate-too-many-workers approach.
I guess, my problem is that I don't agree with that as a goal in an of
itself. If you actually want to run your query quickly, you *want* 20
workers here. The issues of backend startup overhead (already via
parallel_setup_cost), concurrency and such cost should be modelled, not
burried in a formula the user can't change. If we want to make it less
and less likely to start more workers we should make that configurable,
not the default.
I think there's some precedent taken from the parallel seqscan case,
that's not actually applicable here. Parallel seqscans have a good
amount of shared state, both on the kernel and pg side, and that shared
state reduces gains of increasing the number of workers. But with
non-partial scans such shared state largely doesn't exist.
especially if we get partitionwise joins.
About that I am not sure, because we already have support for parallel
joins, so wouldn't the join subpaths corresponding to all of the
partitions be partial paths ? I may be wrong about that.
We'll probably generate both, and then choose the cheaper one. The
startup cost for partitionwise joins should usually be a lot cheaper
(because e.g. for hashtables we'll generate smaller hashtables), and we
should have less cost of concurrency.
But if the subplans are foreign scans, then yes all would be
non-partial plans. This may provoke off-topic discussion, but here
instead of assigning so many workers to all these foreign plans and
all those workers waiting for the results, a single asynchronous
execution node (which is still in the making) would be desirable
because it would do the job of all these workers.
That's something that probably shouldn't be modelled throug a parallel
append, I agree - it shouldn't be too hard to develop a costing model
for that however.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 7 April 2017 at 20:35, Andres Freund <andres@anarazel.de> wrote:
But for costs such as (4, 4, 4, .... 20 times), the logic would give
us 20 workers because we want to finish the Append in 4 time units;
and this what we want to avoid when we go with
don't-allocate-too-many-workers approach.I guess, my problem is that I don't agree with that as a goal in an of
itself. If you actually want to run your query quickly, you *want* 20
workers here. The issues of backend startup overhead (already via
parallel_setup_cost), concurrency and such cost should be modelled, not
burried in a formula the user can't change. If we want to make it less
and less likely to start more workers we should make that configurable,
not the default.
I think there's some precedent taken from the parallel seqscan case,
that's not actually applicable here. Parallel seqscans have a good
amount of shared state, both on the kernel and pg side, and that shared
state reduces gains of increasing the number of workers. But with
non-partial scans such shared state largely doesn't exist.
After searching through earlier mails about parallel scan, I am not
sure whether the shared state was considered to be a potential factor
that might reduce parallel query gains, when deciding the calculation
for number of workers for a parallel seq scan. I mean even today if we
allocate 10 workers as against a calculated 4 workers count for a
parallel seq scan, they might help. I think it's just that we don't
know if they would *always* help or it would regress sometimes.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Apr 4, 2017 at 12:37 PM, Amit Khandekar <amitdkhan.pg@gmail.com>
wrote:
Attached is an updated patch v13 that has some comments changed as per
your review, and also rebased on latest master.
This is not applicable on the latest head i.e. commit
-- 08aed6604de2e6a9f4d499818d7c641cbf5eb9f7, looks like need a rebasing.
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/
On 30 June 2017 at 15:10, Rafia Sabih <rafia.sabih@enterprisedb.com> wrote:
On Tue, Apr 4, 2017 at 12:37 PM, Amit Khandekar <amitdkhan.pg@gmail.com>
wrote:Attached is an updated patch v13 that has some comments changed as per
your review, and also rebased on latest master.This is not applicable on the latest head i.e. commit --
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7, looks like need a rebasing.
Thanks for notifying. Attached is the rebased version of the patch.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
ParallelAppend_v13_rebased.patchapplication/octet-stream; name=ParallelAppend_v13_rebased.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 80d1679..8639922 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3655,6 +3655,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index be3dc67..e7396f3 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -845,7 +845,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1109,6 +1109,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index ce47f1d..26e0a28 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -231,6 +232,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -295,6 +300,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -798,6 +807,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index aae5e3f..539c75e 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -59,9 +59,47 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
+#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
+
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
-static bool exec_append_initialize_next(AppendState *appendstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -72,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -182,10 +229,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -199,6 +246,14 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
TupleTableSlot *
ExecAppend(AppendState *node)
{
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -225,16 +280,31 @@ ExecAppend(AppendState *node)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -272,6 +342,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -291,6 +362,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id, false);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 67ac814..3bf0f85 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index acaf4b5..75761a9 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 3a23f0b..688f16f 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -376,6 +376,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 2988e8b..48e3973 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1579,6 +1579,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index f087ddb..306fc1e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1281,7 +1284,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1317,7 +1324,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1395,23 +1460,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1421,9 +1502,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1476,7 +1557,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1694,6 +1776,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1713,7 +1867,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index eb653cf..5bbc683 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,8 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(List *subpaths, int numpaths,
+ int parallel_workers);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1741,6 +1744,189 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ *
+ * It is the total cost units taken by all the workers to finish all the
+ * non-partial subpaths.
+ * subpaths contains non-partial paths followed by partial paths.
+ * numpaths tells the number of non-partial paths.
+ */
+static Cost
+append_nonpartial_cost(List *subpaths, int numpaths, int parallel_workers)
+{
+ Cost *costarr;
+ int arrlen;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int path_index;
+ int min_index;
+ int max_index;
+
+ if (numpaths == 0)
+ return 0;
+
+ /*
+ * Build the cost array containing costs of first n number of subpaths,
+ * where n = parallel_workers. Also, its size is kept only as long as the
+ * number of subpaths, or parallel_workers, whichever is minimum.
+ */
+ arrlen = Min(parallel_workers, numpaths);
+ costarr = (Cost *) palloc(sizeof(Cost) * arrlen);
+ path_index = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+
+ if (path_index == arrlen)
+ break;
+ costarr[path_index++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = arrlen - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+ int i;
+
+ /* Consider only the non-partial paths */
+ if (path_index++ == numpaths)
+ break;
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+ for (max_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (list_length(subpaths) == 0)
+ return;
+
+ if (!path->parallel_aware)
+ {
+ Path *subpath = (Path *) linitial(subpaths);
+
+ /*
+ * Startup cost of non-parallel-aware Append is the startup cost of
+ * first subpath.
+ */
+ path->startup_cost = subpath->startup_cost;
+
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ else /* parallel-aware */
+ {
+ double max_rows = 0;
+ double nonpartial_rows = 0;
+ int i = 0;
+
+ /* Include the non-partial paths total cost */
+ path->total_cost += append_nonpartial_cost(subpaths,
+ num_nonpartial_subpaths,
+ path->parallel_workers);
+
+ /* Calculate startup cost; also add up all the rows for later use */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (i < path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ if (i < num_nonpartial_subpaths)
+ {
+ nonpartial_rows += subpath->rows;
+
+ /* Also keep track of max rows for any given subpath */
+ max_rows = Max(max_rows, subpath->rows);
+ }
+
+ i++;
+ }
+
+ /*
+ * As an approximation, non-partial rows are calculated as total rows
+ * divided by number of workers. But if there are highly unequal number
+ * of rows across the paths, this figure might not reflect correctly.
+ * So we make a note that it also should not be less than the maximum
+ * of all the path rows.
+ */
+ nonpartial_rows /= path->parallel_workers;
+ path->rows += Max(nonpartial_rows, max_rows);
+
+ /* Calculate partial paths cost. */
+ if (list_length(subpaths) > num_nonpartial_subpaths)
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ for_each_cell(l, list_nth_cell(subpaths, num_nonpartial_subpaths))
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6ee2350..0eee647 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index e589d92..a1297d8 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -203,7 +203,8 @@ 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, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1049,7 +1050,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5269,7 +5271,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5280,6 +5282,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2988c11..7d439d8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3589,8 +3589,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index cf46b74..64479ce 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -576,8 +576,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -688,7 +688,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f2d6385..0b79f0e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,70 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with 3 subplans having per-subplan workers such as (2, 8, 8),
+ * the Append workers should be at least 8, whereas the formula gives 2. In
+ * this case, it seems better to follow the method used for calculating
+ * parallel_workers of an unpartitioned table : log3(table_size). So we
+ * treat a partitioned table as if the data belongs to a single
+ * unpartitioned table, and then derive its workers. So it will be :
+ * logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan workers and
+ * b is some logarithmic base such as 2 or 3. It turns out that this
+ * evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1265,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,43 +1279,50 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered unsorted */
pathnode->partitioned_rels = list_copy(partitioned_rels);
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 82a1cf5..f2770fa 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -494,7 +494,7 @@ RegisterLWLockTranches(void)
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 64;
+ LWLockTranchesAllocated = 128;
LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(char *));
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 82e54c0..2dba157 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -910,6 +910,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 2b1ebb7..dbf0cb3 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -294,6 +294,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index ee0b6ad..d47163b 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,11 +14,15 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern TupleTableSlot *ExecAppend(AppendState *node);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 85fac8a..3f8b124 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/queryenvironment.h"
#include "utils/reltrigger.h"
@@ -976,12 +977,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 667d5e2..711db92 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -269,6 +269,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f1a1b24..74da90d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -248,6 +248,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9bae3c6..247cc34 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1167,10 +1167,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1178,6 +1182,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 63feba0..8e66cf0 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -105,6 +106,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 0c0549d..40d31bb 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 3d16132..35adf12 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 35d182d..6ab7cc7 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 3e35e96..f5bb820 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index 70fe971..6cdc009 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index d2d262c..4b07c03 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Wed, Jul 5, 2017 at 7:53 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
This is not applicable on the latest head i.e. commit --
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7, looks like need a rebasing.Thanks for notifying. Attached is the rebased version of the patch.
This again needs a rebase.
(And, hey everybody, it also needs some review!)
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 9 August 2017 at 19:05, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Jul 5, 2017 at 7:53 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
This is not applicable on the latest head i.e. commit --
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7, looks like need a rebasing.Thanks for notifying. Attached is the rebased version of the patch.
This again needs a rebase.
Attached rebased version of the patch. Thanks.
--
Thanks,
-Amit Khandekar
EnterpriseDB Corporation
The Postgres Database Company
Attachments:
ParallelAppend_v13_rebased_2.patchapplication/octet-stream; name=ParallelAppend_v13_rebased_2.patchDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c33d6a0..d844b99 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -3679,6 +3679,20 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallelappend" xreflabel="enable_parallelappend">
+ <term><varname>enable_parallelappend</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallelappend</> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel-aware
+ append plan types. The default is <literal>on</>.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="guc-enable-seqscan" xreflabel="enable_seqscan">
<term><varname>enable_seqscan</varname> (<type>boolean</type>)
<indexterm>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 12d5628..d365d0b 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -845,7 +845,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<tbody>
<row>
- <entry morerows="59"><literal>LWLock</></entry>
+ <entry morerows="60"><literal>LWLock</></entry>
<entry><literal>ShmemIndexLock</></entry>
<entry>Waiting to find or allocate space in shared memory.</entry>
</row>
@@ -1109,6 +1109,11 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting for TBM shared iterator lock.</entry>
</row>
<row>
+ <entry><literal>parallel_append</></entry>
+ <entry>Waiting to choose the next subplan during Parallel Append plan
+ execution.</entry>
+ </row>
+ <row>
<entry morerows="9"><literal>Lock</></entry>
<entry><literal>relation</></entry>
<entry>Waiting to acquire a lock on a relation.</entry>
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index ce47f1d..26e0a28 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -25,6 +25,7 @@
#include "executor/execParallel.h"
#include "executor/executor.h"
+#include "executor/nodeAppend.h"
#include "executor/nodeBitmapHeapscan.h"
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
@@ -231,6 +232,10 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecForeignScanEstimate((ForeignScanState *) planstate,
e->pcxt);
break;
+ case T_AppendState:
+ ExecAppendEstimate((AppendState *) planstate,
+ e->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanEstimate((CustomScanState *) planstate,
e->pcxt);
@@ -295,6 +300,10 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecForeignScanInitializeDSM((ForeignScanState *) planstate,
d->pcxt);
break;
+ case T_AppendState:
+ ExecAppendInitializeDSM((AppendState *) planstate,
+ d->pcxt);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeDSM((CustomScanState *) planstate,
d->pcxt);
@@ -798,6 +807,9 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
ExecForeignScanInitializeWorker((ForeignScanState *) planstate,
toc);
break;
+ case T_AppendState:
+ ExecAppendInitializeWorker((AppendState *) planstate, toc);
+ break;
case T_CustomScanState:
ExecCustomScanInitializeWorker((CustomScanState *) planstate,
toc);
diff --git a/src/backend/executor/nodeAppend.c b/src/backend/executor/nodeAppend.c
index bed9bb8..11f9688 100644
--- a/src/backend/executor/nodeAppend.c
+++ b/src/backend/executor/nodeAppend.c
@@ -60,10 +60,46 @@
#include "executor/execdebug.h"
#include "executor/nodeAppend.h"
#include "miscadmin.h"
+#include "optimizer/cost.h"
+#include "storage/spin.h"
-static TupleTableSlot *ExecAppend(PlanState *pstate);
-static bool exec_append_initialize_next(AppendState *appendstate);
+/*
+ * Shared state for Parallel Append.
+ *
+ * Each backend participating in a Parallel Append has its own
+ * descriptor in backend-private memory, and those objects all contain
+ * a pointer to this structure.
+ */
+typedef struct ParallelAppendDescData
+{
+ LWLock pa_lock; /* mutual exclusion to choose next subplan */
+ int pa_first_plan; /* plan to choose while wrapping around plans */
+ int pa_next_plan; /* next plan to choose by any worker */
+
+ /*
+ * pa_finished : workers currently executing the subplan. A worker which
+ * finishes a subplan should set pa_finished to true, so that no new
+ * worker picks this subplan. For non-partial subplan, a worker which picks
+ * up that subplan should immediately set to true, so as to make sure
+ * there are no more than 1 worker assigned to this subplan.
+ */
+ bool pa_finished[FLEXIBLE_ARRAY_MEMBER];
+} ParallelAppendDescData;
+
+typedef ParallelAppendDescData *ParallelAppendDesc;
+
+/*
+ * Special value of AppendState->as_whichplan for Parallel Append, which
+ * indicates there are no plans left to be executed.
+ */
+#define PA_INVALID_PLAN -1
+static TupleTableSlot *ExecAppend(PlanState *pstate);
+static bool exec_append_seq_next(AppendState *appendstate);
+static bool exec_append_parallel_next(AppendState *state);
+static bool exec_append_leader_next(AppendState *state);
+static int exec_append_get_next_plan(int curplan, int first_plan,
+ int last_plan);
/* ----------------------------------------------------------------
* exec_append_initialize_next
@@ -74,11 +110,20 @@ static bool exec_append_initialize_next(AppendState *appendstate);
* ----------------------------------------------------------------
*/
static bool
-exec_append_initialize_next(AppendState *appendstate)
+exec_append_seq_next(AppendState *appendstate)
{
int whichplan;
/*
+ * Not parallel-aware. Fine, just go on to the next subplan in the
+ * appropriate direction.
+ */
+ if (ScanDirectionIsForward(appendstate->ps.state->es_direction))
+ appendstate->as_whichplan++;
+ else
+ appendstate->as_whichplan--;
+
+ /*
* get information from the append node
*/
whichplan = appendstate->as_whichplan;
@@ -185,10 +230,10 @@ ExecInitAppend(Append *node, EState *estate, int eflags)
appendstate->ps.ps_ProjInfo = NULL;
/*
- * initialize to scan first subplan
+ * Initialize to scan first subplan (but note that we'll override this
+ * later in the case of a parallel append).
*/
appendstate->as_whichplan = 0;
- exec_append_initialize_next(appendstate);
return appendstate;
}
@@ -204,6 +249,14 @@ ExecAppend(PlanState *pstate)
{
AppendState *node = castNode(AppendState, pstate);
+ /*
+ * Check if we are already finished plans from parallel append. This
+ * can happen if all the subplans are finished when this worker
+ * has not even started returning tuples.
+ */
+ if (node->as_padesc && node->as_whichplan == PA_INVALID_PLAN)
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+
for (;;)
{
PlanState *subnode;
@@ -232,16 +285,31 @@ ExecAppend(PlanState *pstate)
}
/*
- * Go on to the "next" subplan in the appropriate direction. If no
- * more subplans, return the empty slot set up for us by
- * ExecInitAppend.
+ * Go on to the "next" subplan. If no more subplans, return the empty
+ * slot set up for us by ExecInitAppend.
*/
- if (ScanDirectionIsForward(node->ps.state->es_direction))
- node->as_whichplan++;
+ if (!node->as_padesc)
+ {
+ /*
+ * This is Parallel-aware append. Follow it's own logic of choosing
+ * the next subplan.
+ */
+ if (!exec_append_seq_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
else
- node->as_whichplan--;
- if (!exec_append_initialize_next(node))
- return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ {
+ /*
+ * We are done with this subplan. There might be other workers
+ * still processing the last chunk of rows for this same subplan,
+ * but there's no point for new workers to run this subplan, so
+ * mark this subplan as finished.
+ */
+ node->as_padesc->pa_finished[node->as_whichplan] = true;
+
+ if (!exec_append_parallel_next(node))
+ return ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ }
/* Else loop back and try to get a tuple from the new subplan */
}
@@ -279,6 +347,7 @@ void
ExecReScanAppend(AppendState *node)
{
int i;
+ ParallelAppendDesc padesc = node->as_padesc;
for (i = 0; i < node->as_nplans; i++)
{
@@ -298,6 +367,276 @@ ExecReScanAppend(AppendState *node)
if (subnode->chgParam == NULL)
ExecReScan(subnode);
}
+
+ if (padesc)
+ {
+ padesc->pa_first_plan = padesc->pa_next_plan = 0;
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+ }
+
node->as_whichplan = 0;
- exec_append_initialize_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * Parallel Append Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecAppendEstimate
+ *
+ * estimates the space required to serialize Append node.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendEstimate(AppendState *node,
+ ParallelContext *pcxt)
+{
+ node->pappend_len =
+ add_size(offsetof(struct ParallelAppendDescData, pa_finished),
+ sizeof(bool) * node->as_nplans);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, node->pappend_len);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeDSM
+ *
+ * Set up a Parallel Append descriptor.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeDSM(AppendState *node,
+ ParallelContext *pcxt)
+{
+ ParallelAppendDesc padesc;
+
+ padesc = shm_toc_allocate(pcxt->toc, node->pappend_len);
+
+ /*
+ * Just setting all the fields to 0 is enough. The logic of choosing the
+ * next plan in workers will take care of everything else.
+ */
+ memset(padesc, 0, sizeof(ParallelAppendDescData));
+ memset(padesc->pa_finished, 0, sizeof(bool) * node->as_nplans);
+
+ LWLockInitialize(&padesc->pa_lock, LWTRANCHE_PARALLEL_APPEND);
+
+ node->as_padesc = padesc;
+
+ shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id, padesc);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecAppendInitializeWorker
+ *
+ * Copy relevant information from TOC into planstate, and initialize
+ * whatever is required to choose and execute the optimal subplan.
+ * ----------------------------------------------------------------
+ */
+void
+ExecAppendInitializeWorker(AppendState *node, shm_toc *toc)
+{
+ node->as_padesc = shm_toc_lookup(toc, node->ps.plan->plan_node_id, false);
+
+ /* Choose the first subplan to be executed. */
+ (void) exec_append_parallel_next(node);
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_parallel_next
+ *
+ * Determine the next subplan that should be executed. Each worker uses a
+ * shared next_subplan counter index to start looking for unfinished plan,
+ * executes the subplan, then shifts ahead this counter to the next
+ * subplan, so that other workers know which next plan to choose. This
+ * way, workers choose the subplans in round robin order, and thus they
+ * get evenly distributed among the subplans.
+ *
+ * Returns false if and only if all subplans are already finished
+ * processing.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_parallel_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int whichplan;
+ int initial_plan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+ bool found;
+
+ Assert(padesc != NULL);
+
+ /* Backward scan is not supported by parallel-aware plans */
+ Assert(ScanDirectionIsForward(state->ps.state->es_direction));
+
+ /* The parallel leader chooses its next subplan differently */
+ if (!IsParallelWorker())
+ return exec_append_leader_next(state);
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* Make a note of which subplan we have started with */
+ initial_plan = padesc->pa_next_plan;
+
+ /*
+ * Keep going to the next plan until we find an unfinished one. In the
+ * process, also keep track of the first unfinished non-partial subplan. As
+ * the non-partial subplans are taken one by one, the first unfinished
+ * subplan index will shift ahead, so that we don't have to visit the
+ * finished non-partial ones anymore.
+ */
+
+ found = false;
+ for (whichplan = initial_plan; whichplan != PA_INVALID_PLAN;)
+ {
+ /*
+ * Ignore plans that are already done processing. These also include
+ * non-partial subplans which have already been taken by a worker.
+ */
+ if (!padesc->pa_finished[whichplan])
+ {
+ found = true;
+ break;
+ }
+
+ /*
+ * Note: There is a chance that just after the child plan node is
+ * chosen above, some other worker finishes this node and sets
+ * pa_finished to true. In that case, this worker will go ahead and
+ * call ExecProcNode(child_node), which will return NULL tuple since it
+ * is already finished, and then once again this worker will try to
+ * choose next subplan; but this is ok : it's just an extra
+ * "choose_next_subplan" operation.
+ */
+
+ /* Either go to the next plan, or wrap around to the first one */
+ whichplan = exec_append_get_next_plan(whichplan, padesc->pa_first_plan,
+ state->as_nplans - 1);
+
+ /*
+ * If we have wrapped around and returned to the same index again, we
+ * are done scanning.
+ */
+ if (whichplan == initial_plan)
+ break;
+ }
+
+ if (!found)
+ {
+ /*
+ * We didn't find any plan to execute, stop executing, and indicate
+ * the same for other workers to know that there is no next plan.
+ */
+ padesc->pa_next_plan = state->as_whichplan = PA_INVALID_PLAN;
+ }
+ else
+ {
+ /*
+ * If this a non-partial plan, immediately mark it finished, and shift
+ * ahead pa_first_plan.
+ */
+ if (whichplan < first_partial_plan)
+ {
+ padesc->pa_finished[whichplan] = true;
+ padesc->pa_first_plan = whichplan + 1;
+ }
+
+ /*
+ * Set the chosen plan, and the next plan to be picked by other
+ * workers.
+ */
+ state->as_whichplan = whichplan;
+ padesc->pa_next_plan = exec_append_get_next_plan(whichplan,
+ padesc->pa_first_plan,
+ state->as_nplans - 1);
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ return found;
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_leader_next
+ *
+ * To be used only if it's a parallel leader. The backend should scan
+ * backwards from the last plan. This is to prevent it from taking up
+ * the most expensive non-partial plan, i.e. the first subplan.
+ * ----------------------------------------------------------------
+ */
+static bool
+exec_append_leader_next(AppendState *state)
+{
+ ParallelAppendDesc padesc = state->as_padesc;
+ int first_plan;
+ int whichplan;
+ int first_partial_plan = ((Append *)state->ps.plan)->first_partial_plan;
+
+ LWLockAcquire(&padesc->pa_lock, LW_EXCLUSIVE);
+
+ /* The parallel leader should start from the last subplan. */
+ first_plan = padesc->pa_first_plan;
+
+ for (whichplan = state->as_nplans - 1; whichplan >= first_plan;
+ whichplan--)
+ {
+ if (!padesc->pa_finished[whichplan])
+ {
+ /* If this a non-partial plan, immediately mark it finished */
+ if (whichplan < first_partial_plan)
+ padesc->pa_finished[whichplan] = true;
+
+ break;
+ }
+ }
+
+ LWLockRelease(&padesc->pa_lock);
+
+ /* Return false only if we didn't find any plan to execute */
+ if (whichplan < first_plan)
+ {
+ state->as_whichplan = PA_INVALID_PLAN;
+ return false;
+ }
+ else
+ {
+ state->as_whichplan = whichplan;
+ return true;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * exec_append_get_next_plan
+ *
+ * Either go to the next index, or wrap around to the first unfinished one.
+ * Returns this next index. While wrapping around, if the first unfinished
+ * one itself is past the last plan, returns PA_INVALID_PLAN.
+ * ----------------------------------------------------------------
+ */
+static int
+exec_append_get_next_plan(int curplan, int first_plan, int last_plan)
+{
+ Assert(curplan <= last_plan);
+
+ if (curplan < last_plan)
+ return curplan + 1;
+ else
+ {
+ /*
+ * We are already at the last plan. If the first_plan itsef is the last
+ * plan or if it is past the last plan, that means there is no next
+ * plan remaining. Return Invalid.
+ */
+ if (first_plan >= last_plan)
+ return PA_INVALID_PLAN;
+
+ return first_plan;
+ }
}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 45a04b0..dd23eae 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -242,6 +242,7 @@ _copyAppend(const Append *from)
*/
COPY_NODE_FIELD(partitioned_rels);
COPY_NODE_FIELD(appendplans);
+ COPY_SCALAR_FIELD(first_partial_plan);
return newnode;
}
diff --git a/src/backend/nodes/list.c b/src/backend/nodes/list.c
index acaf4b5..75761a9 100644
--- a/src/backend/nodes/list.c
+++ b/src/backend/nodes/list.c
@@ -1250,6 +1250,45 @@ list_copy_tail(const List *oldlist, int nskip)
}
/*
+ * Sort a list using qsort. A sorted list is built but the cells of the original
+ * list are re-used. Caller has to pass a copy of the list if the original list
+ * needs to be untouched. Effectively, the comparator function is passed
+ * pointers to ListCell* pointers.
+ */
+List *
+list_qsort(const List *list, list_qsort_comparator cmp)
+{
+ ListCell *cell;
+ int i;
+ int len = list_length(list);
+ ListCell **list_arr;
+ List *new_list;
+
+ if (len == 0)
+ return NIL;
+
+ i = 0;
+ list_arr = palloc(sizeof(ListCell *) * len);
+ foreach(cell, list)
+ list_arr[i++] = cell;
+
+ qsort(list_arr, len, sizeof(ListCell *), cmp);
+
+ new_list = (List *) palloc(sizeof(List));
+ new_list->type = T_List;
+ new_list->length = len;
+ new_list->head = list_arr[0];
+ new_list->tail = list_arr[len-1];
+
+ for (i = 0; i < len-1; i++)
+ list_arr[i]->next = list_arr[i+1];
+
+ list_arr[len-1]->next = NULL;
+ pfree(list_arr);
+ return new_list;
+}
+
+/*
* Temporary compatibility functions
*
* In order to avoid warnings for these function definitions, we need
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 379d92a..167a28b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -394,6 +394,7 @@ _outAppend(StringInfo str, const Append *node)
WRITE_NODE_FIELD(partitioned_rels);
WRITE_NODE_FIELD(appendplans);
+ WRITE_INT_FIELD(first_partial_plan);
}
static void
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 86c811d..51210b5 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1594,6 +1594,7 @@ _readAppend(void)
READ_NODE_FIELD(partitioned_rels);
READ_NODE_FIELD(appendplans);
+ READ_INT_FIELD(first_partial_plan);
READ_DONE();
}
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index f087ddb..306fc1e 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -101,6 +101,9 @@ static Path *get_cheapest_parameterized_child_path(PlannerInfo *root,
RelOptInfo *rel,
Relids required_outer);
static List *accumulate_append_subpath(List *subpaths, Path *path);
+static List *accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths);
static void set_subquery_pathlist(PlannerInfo *root, RelOptInfo *rel,
Index rti, RangeTblEntry *rte);
static void set_function_pathlist(PlannerInfo *root, RelOptInfo *rel,
@@ -1281,7 +1284,11 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
List *subpaths = NIL;
bool subpaths_valid = true;
List *partial_subpaths = NIL;
+ List *pa_partial_subpaths = NIL;
+ List *pa_nonpartial_subpaths = NIL;
bool partial_subpaths_valid = true;
+ bool pa_subpaths_valid = enable_parallelappend;
+ bool pa_all_partial_subpaths = enable_parallelappend;
List *all_child_pathkeys = NIL;
List *all_child_outers = NIL;
ListCell *l;
@@ -1317,7 +1324,65 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
else
subpaths_valid = false;
- /* Same idea, but for a partial plan. */
+ /* Same idea, but for a parallel append path. */
+ if (pa_subpaths_valid && enable_parallelappend)
+ {
+ Path *chosen_path = NULL;
+ Path *cheapest_partial_path = NULL;
+ Path *cheapest_parallel_safe_path = NULL;
+
+ /*
+ * Extract the cheapest unparameterized, parallel-safe one among
+ * the child paths.
+ */
+ cheapest_parallel_safe_path =
+ get_cheapest_parallel_safe_total_inner(childrel->pathlist);
+
+ /* Get the cheapest partial path */
+ if (childrel->partial_pathlist != NIL)
+ cheapest_partial_path = linitial(childrel->partial_pathlist);
+
+ if (!cheapest_parallel_safe_path && !cheapest_partial_path)
+ {
+ /*
+ * This child rel neither has a partial path, nor has a
+ * parallel-safe path. Drop the idea for parallel append.
+ */
+ pa_subpaths_valid = false;
+ }
+ else if (cheapest_partial_path && cheapest_parallel_safe_path)
+ {
+ /* Both are valid. Choose the cheaper out of the two */
+ if (cheapest_parallel_safe_path->total_cost <
+ cheapest_partial_path->total_cost)
+ chosen_path = cheapest_parallel_safe_path;
+ else
+ chosen_path = cheapest_partial_path;
+ }
+ else
+ {
+ /* Either one is valid. Choose the valid one */
+ chosen_path = cheapest_partial_path ?
+ cheapest_partial_path :
+ cheapest_parallel_safe_path;
+ }
+
+ /* If we got a valid path, add it */
+ if (chosen_path)
+ {
+ pa_partial_subpaths =
+ accumulate_partialappend_subpath(
+ pa_partial_subpaths,
+ chosen_path,
+ chosen_path == cheapest_partial_path,
+ &pa_nonpartial_subpaths);
+ }
+
+ if (chosen_path && chosen_path != cheapest_partial_path)
+ pa_all_partial_subpaths = false;
+ }
+
+ /* Same idea, but for a non-parallel partial plan. */
if (childrel->partial_pathlist != NIL)
partial_subpaths = accumulate_append_subpath(partial_subpaths,
linitial(childrel->partial_pathlist));
@@ -1395,23 +1460,39 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
* if we have zero or one live subpath due to constraint exclusion.)
*/
if (subpaths_valid)
- add_path(rel, (Path *) create_append_path(rel, subpaths, NULL, 0,
+ add_path(rel, (Path *) create_append_path(rel, subpaths, NIL,
+ NULL, 0, false,
partitioned_rels));
+ /* Consider parallel append path. */
+ if (pa_subpaths_valid)
+ {
+ AppendPath *appendpath;
+ int parallel_workers;
+
+ parallel_workers = get_append_num_workers(pa_partial_subpaths,
+ pa_nonpartial_subpaths);
+ appendpath = create_append_path(rel, pa_nonpartial_subpaths,
+ pa_partial_subpaths,
+ NULL, parallel_workers, true,
+ partitioned_rels);
+ add_partial_path(rel, (Path *) appendpath);
+ }
+
/*
- * Consider an append of partial unordered, unparameterized partial paths.
+ * Consider non-parallel partial append path. But if the parallel append
+ * path is made out of all partial subpaths, don't create another partial
+ * path; we will keep only the parallel append path in that case.
*/
- if (partial_subpaths_valid)
+ if (partial_subpaths_valid && !pa_all_partial_subpaths)
{
AppendPath *appendpath;
ListCell *lc;
int parallel_workers = 0;
/*
- * Decide on the number of workers to request for this append path.
- * For now, we just use the maximum value from among the members. It
- * might be useful to use a higher number if the Append node were
- * smart enough to spread out the workers, but it currently isn't.
+ * To decide the number of workers, just use the maximum value from
+ * among the children.
*/
foreach(lc, partial_subpaths)
{
@@ -1421,9 +1502,9 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
}
Assert(parallel_workers > 0);
- /* Generate a partial append path. */
- appendpath = create_append_path(rel, partial_subpaths, NULL,
- parallel_workers, partitioned_rels);
+ appendpath = create_append_path(rel, NIL, partial_subpaths,
+ NULL, parallel_workers, false,
+ partitioned_rels);
add_partial_path(rel, (Path *) appendpath);
}
@@ -1476,7 +1557,8 @@ add_paths_to_append_rel(PlannerInfo *root, RelOptInfo *rel,
if (subpaths_valid)
add_path(rel, (Path *)
- create_append_path(rel, subpaths, required_outer, 0,
+ create_append_path(rel, subpaths, NIL,
+ required_outer, 0, false,
partitioned_rels));
}
}
@@ -1694,6 +1776,78 @@ accumulate_append_subpath(List *subpaths, Path *path)
}
/*
+ * accumulate_partialappend_subpath:
+ * Add a subpath to the list being built for a partial Append.
+ *
+ * This is same as accumulate_append_subpath, except that two separate lists
+ * are created, one containing only partial subpaths, and the other containing
+ * only non-partial subpaths. Also, the non-partial paths are kept ordered
+ * by descending total cost.
+ *
+ * is_partial is true if the subpath being added is a partial subpath.
+ */
+static List *
+accumulate_partialappend_subpath(List *partial_subpaths,
+ Path *subpath, bool is_partial,
+ List **nonpartial_subpaths)
+{
+ /* list_copy is important here to avoid sharing list substructure */
+
+ if (IsA(subpath, AppendPath))
+ {
+ AppendPath *apath = (AppendPath *) subpath;
+ List *apath_partial_paths;
+ List *apath_nonpartial_paths;
+
+ /* Split the Append subpaths into partial and non-partial paths */
+ apath_nonpartial_paths = list_truncate(list_copy(apath->subpaths),
+ apath->first_partial_path);
+ apath_partial_paths = list_copy_tail(apath->subpaths,
+ apath->first_partial_path);
+
+ /* Add non-partial subpaths, if any. */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(apath_nonpartial_paths));
+
+ /* Add partial subpaths, if any. */
+ return list_concat(partial_subpaths, apath_partial_paths);
+ }
+ else if (IsA(subpath, MergeAppendPath))
+ {
+ MergeAppendPath *mpath = (MergeAppendPath *) subpath;
+
+ /*
+ * If at all MergeAppend is partial, all its child plans have to be
+ * partial : we don't currently support a mix of partial and
+ * non-partial MergeAppend subpaths.
+ */
+ if (is_partial)
+ return list_concat(partial_subpaths, list_copy(mpath->subpaths));
+ else
+ {
+ /*
+ * Since MergePath itself is non-partial, treat all its subpaths
+ * non-partial.
+ */
+ *nonpartial_subpaths = list_concat(*nonpartial_subpaths,
+ list_copy(mpath->subpaths));
+ return partial_subpaths;
+ }
+ }
+ else
+ {
+ /* Just add it to the right list depending upon whether it's partial */
+ if (is_partial)
+ return lappend(partial_subpaths, subpath);
+ else
+ {
+ *nonpartial_subpaths = lappend(*nonpartial_subpaths, subpath);
+ return partial_subpaths;
+ }
+ }
+}
+
+/*
* set_dummy_rel_pathlist
* Build a dummy path for a relation that's been excluded by constraints
*
@@ -1713,7 +1867,8 @@ set_dummy_rel_pathlist(RelOptInfo *rel)
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/*
* We set the cheapest path immediately, to ensure that IS_DUMMY_REL()
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index b35acb7..fe677a3 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -127,6 +127,7 @@ bool enable_material = true;
bool enable_mergejoin = true;
bool enable_hashjoin = true;
bool enable_gathermerge = true;
+bool enable_parallelappend = true;
typedef struct
{
@@ -159,6 +160,8 @@ static Selectivity get_foreign_key_join_selectivity(PlannerInfo *root,
Relids inner_relids,
SpecialJoinInfo *sjinfo,
List **restrictlist);
+static Cost append_nonpartial_cost(List *subpaths, int numpaths,
+ int parallel_workers);
static void set_rel_width(PlannerInfo *root, RelOptInfo *rel);
static double relation_byte_size(double tuples, int width);
static double page_size(double tuples, int width);
@@ -1741,6 +1744,189 @@ cost_sort(Path *path, PlannerInfo *root,
}
/*
+ * append_nonpartial_cost
+ * Determines and returns the cost of non-partial paths of Append node.
+ *
+ * It is the total cost units taken by all the workers to finish all the
+ * non-partial subpaths.
+ * subpaths contains non-partial paths followed by partial paths.
+ * numpaths tells the number of non-partial paths.
+ */
+static Cost
+append_nonpartial_cost(List *subpaths, int numpaths, int parallel_workers)
+{
+ Cost *costarr;
+ int arrlen;
+ ListCell *l;
+ ListCell *cell;
+ int i;
+ int path_index;
+ int min_index;
+ int max_index;
+
+ if (numpaths == 0)
+ return 0;
+
+ /*
+ * Build the cost array containing costs of first n number of subpaths,
+ * where n = parallel_workers. Also, its size is kept only as long as the
+ * number of subpaths, or parallel_workers, whichever is minimum.
+ */
+ arrlen = Min(parallel_workers, numpaths);
+ costarr = (Cost *) palloc(sizeof(Cost) * arrlen);
+ path_index = 0;
+ foreach(cell, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(cell);
+
+ if (path_index == arrlen)
+ break;
+ costarr[path_index++] = subpath->total_cost;
+ }
+
+ /*
+ * Since the subpaths are non-partial paths, the array is initially sorted
+ * by decreasing cost. So choose the last one for the index with minimum
+ * cost.
+ */
+ min_index = arrlen - 1;
+
+ /*
+ * For each of the remaining subpaths, add its cost to the array element
+ * with minimum cost.
+ */
+ for_each_cell(l, cell)
+ {
+ Path *subpath = (Path *) lfirst(l);
+ int i;
+
+ /* Consider only the non-partial paths */
+ if (path_index++ == numpaths)
+ break;
+
+ costarr[min_index] += subpath->total_cost;
+
+ /* Update the new min cost array index */
+ for (min_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] < costarr[min_index])
+ min_index = i;
+ }
+ }
+
+ /* Return the highest cost from the array */
+ for (max_index = i = 0; i < arrlen; i++)
+ {
+ if (costarr[i] > costarr[max_index])
+ max_index = i;
+ }
+
+ return costarr[max_index];
+}
+
+/*
+ * cost_append
+ * Determines and returns the cost of an Append node.
+ *
+ * We charge nothing extra for the Append itself, which perhaps is too
+ * optimistic, but since it doesn't do any selection or projection, it is a
+ * pretty cheap node.
+ */
+void
+cost_append(Path *path, List *subpaths, int num_nonpartial_subpaths)
+{
+ ListCell *l;
+
+ path->rows = 0;
+ path->startup_cost = 0;
+ path->total_cost = 0;
+
+ if (list_length(subpaths) == 0)
+ return;
+
+ if (!path->parallel_aware)
+ {
+ Path *subpath = (Path *) linitial(subpaths);
+
+ /*
+ * Startup cost of non-parallel-aware Append is the startup cost of
+ * first subpath.
+ */
+ path->startup_cost = subpath->startup_cost;
+
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ else /* parallel-aware */
+ {
+ double max_rows = 0;
+ double nonpartial_rows = 0;
+ int i = 0;
+
+ /* Include the non-partial paths total cost */
+ path->total_cost += append_nonpartial_cost(subpaths,
+ num_nonpartial_subpaths,
+ path->parallel_workers);
+
+ /* Calculate startup cost; also add up all the rows for later use */
+ foreach(l, subpaths)
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ /*
+ * Append would start returning tuples when the child node having
+ * lowest startup cost is done setting up. We consider only the
+ * first few subplans that immediately get a worker assigned.
+ */
+ if (i < path->parallel_workers)
+ {
+ path->startup_cost = Min(path->startup_cost,
+ subpath->startup_cost);
+ }
+
+ if (i < num_nonpartial_subpaths)
+ {
+ nonpartial_rows += subpath->rows;
+
+ /* Also keep track of max rows for any given subpath */
+ max_rows = Max(max_rows, subpath->rows);
+ }
+
+ i++;
+ }
+
+ /*
+ * As an approximation, non-partial rows are calculated as total rows
+ * divided by number of workers. But if there are highly unequal number
+ * of rows across the paths, this figure might not reflect correctly.
+ * So we make a note that it also should not be less than the maximum
+ * of all the path rows.
+ */
+ nonpartial_rows /= path->parallel_workers;
+ path->rows += Max(nonpartial_rows, max_rows);
+
+ /* Calculate partial paths cost. */
+ if (list_length(subpaths) > num_nonpartial_subpaths)
+ {
+ /* Compute rows and costs as sums of subplan rows and costs. */
+ for_each_cell(l, list_nth_cell(subpaths, num_nonpartial_subpaths))
+ {
+ Path *subpath = (Path *) lfirst(l);
+
+ path->rows += subpath->rows;
+ path->total_cost += subpath->total_cost;
+ }
+ }
+ }
+}
+
+/*
* cost_merge_append
* Determines and returns the cost of a MergeAppend node.
*
diff --git a/src/backend/optimizer/path/joinrels.c b/src/backend/optimizer/path/joinrels.c
index 6ee2350..0eee647 100644
--- a/src/backend/optimizer/path/joinrels.c
+++ b/src/backend/optimizer/path/joinrels.c
@@ -1217,7 +1217,8 @@ mark_dummy_rel(RelOptInfo *rel)
rel->partial_pathlist = NIL;
/* Set up the dummy path */
- add_path(rel, (Path *) create_append_path(rel, NIL, NULL, 0, NIL));
+ add_path(rel, (Path *) create_append_path(rel, NIL, NIL, NULL,
+ 0, false, NIL));
/* Set or update cheapest_total_path and related fields */
set_cheapest(rel);
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 5c934f2..9c7a6d6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -203,7 +203,8 @@ 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, List *tlist, List *partitioned_rels);
+static Append *make_append(List *appendplans, int first_partial_plan,
+ List *tlist, List *partitioned_rels);
static RecursiveUnion *make_recursive_union(List *tlist,
Plan *lefttree,
Plan *righttree,
@@ -1049,7 +1050,8 @@ create_append_plan(PlannerInfo *root, AppendPath *best_path)
* parent-rel Vars it'll be asked to emit.
*/
- plan = make_append(subplans, tlist, best_path->partitioned_rels);
+ plan = make_append(subplans, best_path->first_partial_path,
+ tlist, best_path->partitioned_rels);
copy_generic_path_info(&plan->plan, (Path *) best_path);
@@ -5270,7 +5272,7 @@ make_foreignscan(List *qptlist,
}
static Append *
-make_append(List *appendplans, List *tlist, List *partitioned_rels)
+make_append(List *appendplans, int first_partial_plan, List *tlist, List *partitioned_rels)
{
Append *node = makeNode(Append);
Plan *plan = &node->plan;
@@ -5281,6 +5283,7 @@ make_append(List *appendplans, List *tlist, List *partitioned_rels)
plan->righttree = NULL;
node->partitioned_rels = partitioned_rels;
node->appendplans = appendplans;
+ node->first_partial_plan = first_partial_plan;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2988c11..7d439d8 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3589,8 +3589,10 @@ create_grouping_paths(PlannerInfo *root,
path = (Path *)
create_append_path(grouped_rel,
paths,
+ NIL,
NULL,
0,
+ false,
NIL);
path->pathtarget = target;
}
diff --git a/src/backend/optimizer/prep/prepunion.c b/src/backend/optimizer/prep/prepunion.c
index cf46b74..64479ce 100644
--- a/src/backend/optimizer/prep/prepunion.c
+++ b/src/backend/optimizer/prep/prepunion.c
@@ -576,8 +576,8 @@ generate_union_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
-
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
@@ -688,7 +688,8 @@ generate_nonunion_path(SetOperationStmt *op, PlannerInfo *root,
/*
* Append the child results together.
*/
- path = (Path *) create_append_path(result_rel, pathlist, NULL, 0, NIL);
+ path = (Path *) create_append_path(result_rel, pathlist, NIL,
+ NULL, 0, false, NIL);
/* We have to manually jam the right tlist into the path; ick */
path->pathtarget = create_pathtarget(root, tlist);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f2d6385..0b79f0e 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -46,6 +46,7 @@ typedef enum
#define STD_FUZZ_FACTOR 1.01
static List *translate_sub_tlist(List *tlist, int relid);
+static int append_total_cost_compare(const void *a, const void *b);
/*****************************************************************************
@@ -1193,6 +1194,70 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
}
/*
+ * get_append_num_workers
+ * Return the number of workers to request for partial append path.
+ */
+int
+get_append_num_workers(List *partial_subpaths, List *nonpartial_subpaths)
+{
+ ListCell *lc;
+ double log2w;
+ int num_workers;
+ int max_per_plan_workers;
+
+ /*
+ * log2(number_of_subpaths)+1 formula seems to give an appropriate number of
+ * workers for Append path either having high number of children (> 100) or
+ * having all non-partial subpaths or subpaths with 1-2 parallel_workers.
+ * Whereas, if the subpaths->parallel_workers is high, this formula is not
+ * suitable, because it does not take into account per-subpath workers.
+ * For e.g., with 3 subplans having per-subplan workers such as (2, 8, 8),
+ * the Append workers should be at least 8, whereas the formula gives 2. In
+ * this case, it seems better to follow the method used for calculating
+ * parallel_workers of an unpartitioned table : log3(table_size). So we
+ * treat a partitioned table as if the data belongs to a single
+ * unpartitioned table, and then derive its workers. So it will be :
+ * logb(b^w1 + b^w2 + b^w3) where w1, w2.. are per-subplan workers and
+ * b is some logarithmic base such as 2 or 3. It turns out that this
+ * evaluates to a value just a bit greater than max(w1,w2, w3). So, we
+ * just use the maximum of workers formula. But this formula gives too few
+ * workers when all paths have single worker (meaning they are non-partial)
+ * For e.g. with workers : (1, 1, 1, 1, 1, 1), it is better to allocate 3
+ * workers, whereas this method allocates only 1.
+ * So we use whichever method that gives higher number of workers.
+ */
+
+ /* Get log2(num_subpaths) */
+ log2w = fls(list_length(partial_subpaths) +
+ list_length(nonpartial_subpaths));
+
+ /* Avoid further calculations if we already crossed max workers limit */
+ if (max_parallel_workers_per_gather <= log2w + 1)
+ return max_parallel_workers_per_gather;
+
+
+ /*
+ * Get the parallel_workers value of the partial subpath having the highest
+ * parallel_workers.
+ */
+ max_per_plan_workers = 1;
+ foreach(lc, partial_subpaths)
+ {
+ Path *subpath = lfirst(lc);
+ max_per_plan_workers = Max(max_per_plan_workers,
+ subpath->parallel_workers);
+ }
+
+ /* Choose the higher of the results of the two formulae */
+ num_workers = rint(Max(log2w, max_per_plan_workers) + 1);
+
+ /* In no case use more than max_parallel_workers_per_gather workers. */
+ num_workers = Min(num_workers, max_parallel_workers_per_gather);
+
+ return num_workers;
+}
+
+/*
* create_append_path
* Creates a path corresponding to an Append plan, returning the
* pathnode.
@@ -1200,8 +1265,11 @@ create_tidscan_path(PlannerInfo *root, RelOptInfo *rel, List *tidquals,
* Note that we must handle subpaths = NIL, representing a dummy access path.
*/
AppendPath *
-create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
- int parallel_workers, List *partitioned_rels)
+create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels)
{
AppendPath *pathnode = makeNode(AppendPath);
ListCell *l;
@@ -1211,43 +1279,50 @@ create_append_path(RelOptInfo *rel, List *subpaths, Relids required_outer,
pathnode->path.pathtarget = rel->reltarget;
pathnode->path.param_info = get_appendrel_parampathinfo(rel,
required_outer);
- pathnode->path.parallel_aware = false;
+ pathnode->path.parallel_aware = (parallel_aware && parallel_workers > 0);
pathnode->path.parallel_safe = rel->consider_parallel;
pathnode->path.parallel_workers = parallel_workers;
pathnode->path.pathkeys = NIL; /* result is always considered unsorted */
pathnode->partitioned_rels = list_copy(partitioned_rels);
- pathnode->subpaths = subpaths;
- /*
- * We don't bother with inventing a cost_append(), but just do it here.
- *
- * Compute rows and costs as sums of subplan rows and costs. We charge
- * nothing extra for the Append itself, which perhaps is too optimistic,
- * but since it doesn't do any selection or projection, it is a pretty
- * cheap node.
- */
- pathnode->path.rows = 0;
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
+ /* For parallel append, non-partial paths are sorted by descending costs */
+ if (pathnode->path.parallel_aware)
+ subpaths = list_qsort(subpaths, append_total_cost_compare);
+
+ pathnode->first_partial_path = list_length(subpaths);
+ pathnode->subpaths = list_concat(subpaths, partial_subpaths);
+
foreach(l, subpaths)
{
Path *subpath = (Path *) lfirst(l);
- pathnode->path.rows += subpath->rows;
-
- if (l == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
pathnode->path.parallel_safe = pathnode->path.parallel_safe &&
- subpath->parallel_safe;
+ subpath->parallel_safe;
/* All child paths must have same parameterization */
Assert(bms_equal(PATH_REQ_OUTER(subpath), required_outer));
}
+ cost_append(&pathnode->path, pathnode->subpaths,
+ pathnode->first_partial_path);
+
return pathnode;
}
+static int
+append_total_cost_compare(const void *a, const void *b)
+{
+ Path *path1 = (Path *) lfirst(*(ListCell **) a);
+ Path *path2 = (Path *) lfirst(*(ListCell **) b);
+
+ if (path1->total_cost > path2->total_cost)
+ return -1;
+ if (path1->total_cost < path2->total_cost)
+ return 1;
+
+ return 0;
+}
+
/*
* create_merge_append_path
* Creates a path corresponding to a MergeAppend plan, returning the
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 82a1cf5..f2770fa 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -494,7 +494,7 @@ RegisterLWLockTranches(void)
if (LWLockTrancheArray == NULL)
{
- LWLockTranchesAllocated = 64;
+ LWLockTranchesAllocated = 128;
LWLockTrancheArray = (char **)
MemoryContextAllocZero(TopMemoryContext,
LWLockTranchesAllocated * sizeof(char *));
@@ -511,6 +511,7 @@ RegisterLWLockTranches(void)
LWLockRegisterTranche(LWTRANCHE_PARALLEL_QUERY_DSA,
"parallel_query_dsa");
LWLockRegisterTranche(LWTRANCHE_TBM, "tbm");
+ LWLockRegisterTranche(LWTRANCHE_PARALLEL_APPEND, "parallel_append");
/* Register named tranches. */
for (i = 0; i < NamedLWLockTrancheRequests; i++)
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 246fea8..0782aa3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -910,6 +910,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
+ {
+ {"enable_parallelappend", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel append plans."),
+ NULL
+ },
+ &enable_parallelappend,
+ true,
+ NULL, NULL, NULL
+ },
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index df5d2f3..0a079b2 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -297,6 +297,7 @@
#enable_material = on
#enable_mergejoin = on
#enable_nestloop = on
+#enable_parallelappend = on
#enable_seqscan = on
#enable_sort = on
#enable_tidscan = on
diff --git a/src/include/executor/nodeAppend.h b/src/include/executor/nodeAppend.h
index 4e38a13..7d9e881 100644
--- a/src/include/executor/nodeAppend.h
+++ b/src/include/executor/nodeAppend.h
@@ -14,10 +14,14 @@
#ifndef NODEAPPEND_H
#define NODEAPPEND_H
+#include "access/parallel.h"
#include "nodes/execnodes.h"
extern AppendState *ExecInitAppend(Append *node, EState *estate, int eflags);
extern void ExecEndAppend(AppendState *node);
extern void ExecReScanAppend(AppendState *node);
+extern void ExecAppendEstimate(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeDSM(AppendState *node, ParallelContext *pcxt);
+extern void ExecAppendInitializeWorker(AppendState *node, shm_toc *toc);
#endif /* NODEAPPEND_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 35c28a6..ea76d4b 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -21,6 +21,7 @@
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/spin.h"
#include "utils/hsearch.h"
#include "utils/queryenvironment.h"
#include "utils/reltrigger.h"
@@ -992,12 +993,15 @@ typedef struct ModifyTableState
* whichplan which plan is being executed (0 .. n-1)
* ----------------
*/
+struct ParallelAppendDescData;
typedef struct AppendState
{
PlanState ps; /* its first field is NodeTag */
PlanState **appendplans; /* array of PlanStates for my inputs */
int as_nplans;
int as_whichplan;
+ struct ParallelAppendDescData *as_padesc; /* parallel coordination info */
+ Size pappend_len; /* size of parallel coordination info */
} AppendState;
/* ----------------
diff --git a/src/include/nodes/pg_list.h b/src/include/nodes/pg_list.h
index 667d5e2..711db92 100644
--- a/src/include/nodes/pg_list.h
+++ b/src/include/nodes/pg_list.h
@@ -269,6 +269,9 @@ extern void list_free_deep(List *list);
extern List *list_copy(const List *list);
extern List *list_copy_tail(const List *list, int nskip);
+typedef int (*list_qsort_comparator) (const void *a, const void *b);
+extern List *list_qsort(const List *list, list_qsort_comparator cmp);
+
/*
* To ease migration to the new list API, a set of compatibility
* macros are provided that reduce the impact of the list API changes
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f1a1b24..74da90d 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -248,6 +248,7 @@ typedef struct Append
/* RT indexes of non-leaf tables in a partition tree */
List *partitioned_rels;
List *appendplans;
+ int first_partial_plan;
} Append;
/* ----------------
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 9bae3c6..247cc34 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -1167,10 +1167,14 @@ typedef struct CustomPath
* AppendPath represents an Append plan, ie, successive execution of
* several member plans.
*
+ * For partial Append, 'subpaths' contains non-partial subpaths followed by
+ * partial subpaths.
+ *
* Note: it is possible for "subpaths" to contain only one, or even no,
* elements. These cases are optimized during create_append_plan.
* In particular, an AppendPath with no subpaths is a "dummy" path that
* is created to represent the case that a relation is provably empty.
+ *
*/
typedef struct AppendPath
{
@@ -1178,6 +1182,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 */
+ int first_partial_path;
} AppendPath;
#define IS_DUMMY_PATH(p) \
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 63feba0..8e66cf0 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -67,6 +67,7 @@ extern bool enable_material;
extern bool enable_mergejoin;
extern bool enable_hashjoin;
extern bool enable_gathermerge;
+extern bool enable_parallelappend;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@@ -105,6 +106,8 @@ extern void cost_sort(Path *path, PlannerInfo *root,
List *pathkeys, Cost input_cost, double tuples, int width,
Cost comparison_cost, int sort_mem,
double limit_tuples);
+extern void cost_append(Path *path, List *subpaths,
+ int num_nonpartial_subpaths);
extern void cost_merge_append(Path *path, PlannerInfo *root,
List *pathkeys, int n_streams,
Cost input_startup_cost, Cost input_total_cost,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 0c0549d..40d31bb 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -14,6 +14,7 @@
#ifndef PATHNODE_H
#define PATHNODE_H
+#include "nodes/bitmapset.h"
#include "nodes/relation.h"
@@ -63,9 +64,13 @@ extern BitmapOrPath *create_bitmap_or_path(PlannerInfo *root,
List *bitmapquals);
extern TidPath *create_tidscan_path(PlannerInfo *root, RelOptInfo *rel,
List *tidquals, Relids required_outer);
-extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths,
- Relids required_outer, int parallel_workers,
- List *partitioned_rels);
+extern int get_append_num_workers(List *partial_subpaths,
+ List *nonpartial_subpaths);
+extern AppendPath *create_append_path(RelOptInfo *rel,
+ List *subpaths, List *partial_subpaths,
+ Relids required_outer,
+ int parallel_workers, bool parallel_aware,
+ List *partitioned_rels);
extern MergeAppendPath *create_merge_append_path(PlannerInfo *root,
RelOptInfo *rel,
List *subpaths,
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index 3d16132..35adf12 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -213,6 +213,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_TBM,
+ LWTRANCHE_PARALLEL_APPEND,
LWTRANCHE_FIRST_USER_DEFINED
} BuiltinTrancheIds;
diff --git a/src/test/regress/expected/inherit.out b/src/test/regress/expected/inherit.out
index 1fa9650..7a5b3c7 100644
--- a/src/test/regress/expected/inherit.out
+++ b/src/test/regress/expected/inherit.out
@@ -1382,6 +1382,7 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
QUERY PLAN
------------------------------------------------------------------
@@ -1448,6 +1449,7 @@ select min(1-id) from matest0;
(1 row)
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table matest1
diff --git a/src/test/regress/expected/select_parallel.out b/src/test/regress/expected/select_parallel.out
index 3e35e96..f5bb820 100644
--- a/src/test/regress/expected/select_parallel.out
+++ b/src/test/regress/expected/select_parallel.out
@@ -11,15 +11,16 @@ set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
QUERY PLAN
-----------------------------------------------------
Finalize Aggregate
-> Gather
- Workers Planned: 1
+ Workers Planned: 4
-> Partial Aggregate
- -> Append
+ -> Parallel Append
-> Parallel Seq Scan on a_star
-> Parallel Seq Scan on b_star
-> Parallel Seq Scan on c_star
@@ -28,12 +29,40 @@ explain (costs off)
-> Parallel Seq Scan on f_star
(11 rows)
-select count(*) from a_star;
- count
--------
- 50
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
+(1 row)
+
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+ QUERY PLAN
+-----------------------------------------------------
+ Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Append
+ -> Seq Scan on d_star
+ -> Seq Scan on c_star
+ -> Parallel Seq Scan on a_star
+ -> Parallel Seq Scan on b_star
+ -> Parallel Seq Scan on e_star
+ -> Parallel Seq Scan on f_star
+(11 rows)
+
+select round(avg(aa)), sum(aa) from a_star;
+ round | sum
+-------+-----
+ 14 | 355
(1 row)
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
explain (verbose, costs off)
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 568b783..97a9843 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -70,21 +70,22 @@ select count(*) >= 0 as ok from pg_prepared_xacts;
-- This is to record the prevailing planner enable_foo settings during
-- a regression test run.
select name, setting from pg_settings where name like 'enable%';
- name | setting
-----------------------+---------
- enable_bitmapscan | on
- enable_gathermerge | on
- enable_hashagg | on
- enable_hashjoin | on
- enable_indexonlyscan | on
- enable_indexscan | on
- enable_material | on
- enable_mergejoin | on
- enable_nestloop | on
- enable_seqscan | on
- enable_sort | on
- enable_tidscan | on
-(12 rows)
+ name | setting
+-----------------------+---------
+ enable_bitmapscan | on
+ enable_gathermerge | on
+ enable_hashagg | on
+ enable_hashjoin | on
+ enable_indexonlyscan | on
+ enable_indexscan | on
+ enable_material | on
+ enable_mergejoin | on
+ enable_nestloop | on
+ enable_parallelappend | on
+ enable_seqscan | on
+ enable_sort | on
+ enable_tidscan | on
+(13 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail
diff --git a/src/test/regress/sql/inherit.sql b/src/test/regress/sql/inherit.sql
index c96580c..60ac387 100644
--- a/src/test/regress/sql/inherit.sql
+++ b/src/test/regress/sql/inherit.sql
@@ -491,11 +491,13 @@ select min(1-id) from matest0;
reset enable_indexscan;
set enable_seqscan = off; -- plan with fewest seqscans should be merge
+set enable_parallelappend = off; -- Don't let parallel-append interfere
explain (verbose, costs off) select * from matest0 order by 1-id;
select * from matest0 order by 1-id;
explain (verbose, costs off) select min(1-id) from matest0;
select min(1-id) from matest0;
reset enable_seqscan;
+reset enable_parallelappend;
drop table matest0 cascade;
diff --git a/src/test/regress/sql/select_parallel.sql b/src/test/regress/sql/select_parallel.sql
index d2d262c..4b07c03 100644
--- a/src/test/regress/sql/select_parallel.sql
+++ b/src/test/regress/sql/select_parallel.sql
@@ -15,9 +15,18 @@ set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
+-- test Parallel Append.
explain (costs off)
- select count(*) from a_star;
-select count(*) from a_star;
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+-- Mix of partial and non-partial subplans.
+alter table c_star set (parallel_workers = 0);
+alter table d_star set (parallel_workers = 0);
+explain (costs off)
+ select round(avg(aa)), sum(aa) from a_star;
+select round(avg(aa)), sum(aa) from a_star;
+alter table c_star reset (parallel_workers);
+alter table d_star reset (parallel_workers);
-- test that parallel_restricted function doesn't run in worker
alter table tenk1 set (parallel_workers = 4);
On Thu, Aug 10, 2017 at 11:04 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
On 9 August 2017 at 19:05, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Jul 5, 2017 at 7:53 AM, Amit Khandekar <amitdkhan.pg@gmail.com> wrote:
This is not applicable on the latest head i.e. commit --
08aed6604de2e6a9f4d499818d7c641cbf5eb9f7, looks like need a rebasing.Thanks for notifying. Attached is the rebased version of the patch.
This again needs a rebase.
Attached rebased version of the patch. Thanks.
I tested this patch for partitioned TPC-H queries along with
partition-wise join patches [1]CAFjFpRfy-YBL6AX3yeO30pAupTMQXgkxDc2P3XBK52QDzGtX5Q@mail.gmail.com. The experimental setup used is as
follows,
Partitions were done on tables lineitem and orders and the partitioned
keys were l_orderkey and o_orderkey respectively. Range partitioning
scheme was used and the total number of partitions for each of the
tables was 17. These experiments are on scale factor 20. Server
parameters are kept as follows,
work_mem = 1GB
shared_buffers = 10GB
effective_cache_size = 10GB
All the values of time are in seconds
Query | Head | ParallelAppend + PWJ | Patches used by query
Q1 | 395 | 398 | only PA
Q3 | 130 | 90 | only PA
Q4 | 244 | 12 | PA and PWJ, time by only PWJ - 41
Q5 | 123 | 77 | PA only
Q6 | 29 | 12 | PA only
Q7 | 134 | 88 | PA only
Q9 | 1051 | 1135 | PA only
Q10 | 111 | 70 | PA and PWJ, time by only PWJ - 89
Q12 | 114 | 70 | PA and PWJ, time by only PWJ - 100
Q14 | 13 | 12 | PA only
Q18 | 508 | 489 | PA only
Q21 | 649 | 163 | PA only
To conclude, the patch is working good for the benchmark with no
serious cases of regression atleast at this scale factor and the
improvement in performance is significant. Please find the attached
file for the explain analyse output of the queries.
[1]: CAFjFpRfy-YBL6AX3yeO30pAupTMQXgkxDc2P3XBK52QDzGtX5Q@mail.gmail.com
--
Regards,
Rafia Sabih
EnterpriseDB: http://www.enterprisedb.com/
Attachments:
PA_test.zipapplication/zip; name=PA_test.zipDownload
PK
�FK PA_test/UX ���Y���Y� PK �FK PA_test/.DS_StoreUX ���Y���Y� ��[hU ����6����M;i�4�������X���dcHB7�6v3�36���ugvS
�h�jQAD�Z�R����
EA6}�_T�<3s6������?���2���|s�3KAG^i$��"�iYU?W(�=�N����H���i+��W���/_�|���� k��}��u�Y��c��<�p.)C�G����y~]�{����r�r�q���/Z���;��P@�G���V�}����<>�������t�n�Y�o���YvC�%�u�K���2���
�$'���-�y�C�rc���y8!�����i��a9=���)�9n��jnI���P��3o(�8O�{2E�DN-�� �P�5KKk�f�!��#E+gf���7��Gg��XxK��,�Oj�4���eg�?����O�9���^��k ��lP>Y9(�UHx�x�8(��x���Wv�)Z�4r kk�a
�9�y��[��=�����z~��R4���L^�m�]������\�����0-��fl�V��N���������ne������-�f�U��NV�E:�q���UW_��r�|9
W������3��:4����leTC��#���v���B��c��!O��4\^U-��
���T�|��!����jI:���s*�U�M��C-�**�w����ml���R�����.�U�*!�XC�(�����<��:Q��SS�qV�<��M�;w��-{ ��U���;����vAsr���jgq�w�v
^ �^B�d�$}d���y�<AN�����-���|C�w�G����B~�aT�u��=���hA+�������8#� h�l�q��q<���8N�4��,^��x
��|����5*P���bHqaUJ��8�:������{.aJ6:�������_�q�����\y��@�Y�E��WUo��V+F�X��r��O-�����Z%��\�>���w�E"4�������HK\��(#7B�� ;Q��H�.�N�����w�v�b�*������ED���cF�`��'�����?P�*lgd^��<�v�dn� Hb�q����0�E�esw���c��x��1<���4��I��x/�e��w�.����
�.c�w��W�ZH��4������W%�U9����������,;��d�u��kG�9Q,@��C ��ax�}�������PK=�z�� PK
�FK __MACOSX/UX ���Y���Y� PK
�FK __MACOSX/PA_test/UX ���Y���Y� PK �FK __MACOSX/PA_test/._.DS_StoreUX ���Y���Y� c`cg`b`�MLV�V�P��'���@� W!��8��A�0]� PK���4 x PK
�FK
PA_test/head/UX ���Y���Y� PK 18�J PA_test/head/10_1.outUX ���Yv�vY� �]Y��~���7K) �F���\%�q"+�W*O.�;���K�y��}����sp���
J��I{���>�
0�>���O���w����W���G��������f������f�\h���c��Gg������g�..�W�@<������:�.n���
w�9����.�^�n7���#��������E6�m���|��?����O��-g7����fw����b�/�������9�m�//����b�g�_\\,wt�b�=��~���W��� �=��|�nv�4[�����-K���\^���f�����r�]���M���o���,{�Z�����`��E�����y��9����g�a~����d�|���A?�����E�]�>{�]���M�����jM����W�%^��-�����y��z��}���:?�����v�qA��hw���#$"]="�v�����Z�NT����D�(F�� ]_uO���^���W���~��4a4X����-��hqDi�FvQ��K�H�8'{uv\s��X����7������5����tX�����&{}=[.s"�~���n9�x�W���z� ���S����`�J�9g��"�m
Z�W� �2-��
�ahly) ��/�=�>_�i�=���qy����E����|�N�&sI�*y�\ �"���p���I��h�x)S���Xw�O>����Z �I��E��XT�-�C�5}D�;�F0w�b1�A�[�F�M�b�1�Y�k|X�~�-��C�B3l�C(��\�F����OFZ�wB���wc��:=�(��7��N�|�����u�XF\+B2�yd�LH�%�xGT[��KK�9��U�t*����c���^- ���a��� ��Yx� �����<��x�*f��%C�9����=)��"(6��Pz(���bZc��
��c��y���� �Jn#j�h�6�R�N3 -�t&ly���+{���="v����v�����'M"'���z6G72I/A��^(�CQ�!�)��T�� �Ca�a�CJ�4���s�(y2B��q�����J� �,�)����a��B�^P+�����s����t��O�T�GSa���]�=�bO���,�����3���o�9���:�����do��e�Zf��w��9k�b_C`y,�M�L�#�!�*���x��'M���b�Is|�������Z���z����{�U_�������j�>��������u�o~���X�k����<{�#2ey�#��A����c��hE
�r"��6�)�1
0Z"?��2���&���^{t __�_$�����K |Q�Q2m*=ZJ���:�`)�;,�_D����B5 |�� �FU��i�����>�c2���_ � �N�z�<�C4���y��2��XG+�O|�������
!�������
�+�ku�<�9:�/�/��/�_N|4 ?�� 3����;�DvX|,��1�Q��_�->8�"�)��D]���N��*\W��\�J|5|�|������|���B��5s1�#�+Hh��H�D,=(J��T_���%�.�:@��1����T_���G����N|==�) I |2]�G����2�&!�e���SjQ��2����O_�UU G��g(
3��%�f���&�����t*��1��qY��"YJ�� '��RS`�S%��#��4��1>�=8�Ul���I�o�o��o��~������I���PG�SZ|��:.�h�>����
|�D�$�z$��*�X�2R�4�w��w}�w����u�K��� ��r5�XP[uV��'����m�Xc'
u����t��i��/Kz�eLE[�l�P����]�/$~����o1��/�h���V��4�1A$2WAG(~^>}���[�}V�5�����@M3�l|�i�[��B_��Una��������d*F�d��8kmqOd������UtePL���[iY��5��������m���3�y�$�� �`��*���r�WnQbO��]F/��z����N��)�3�
"��V53l�oI����La��c�Z���_a�ka� �r}�[HV��OP���S,g�2����JKg-����)`y�R�. Dq�-}���+��|���3Q��&�P�
�MU�[!OS����-�Un!Y�>A��Z�(��c���� �S�:!��B��5���i����Z�X��e4M�>����DQ���D��`��*���r�Wn��T�:���d��b\��*D��W[��p��=�!���+�~S����V[7��������vQ��4��0X����-$�����[�$Qwf�������g�DvX� �����n�P��r[m��RZ�w���O��Dk8�uZ���B�:-L\�M��r���)��5Q���dQ���Zs_�����5rT�)"