Parallel INSERT (INTO ... SELECT ...)
Hi Hackers,
Following on from Dilip Kumar's POC patch for allowing parallelism of
the SELECT part of "INSERT INTO ... SELECT ...", I have attached a POC
patch for allowing parallelism of both the INSERT and SELECT parts,
where it can be allowed.
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.
Obviously I've had to update the planner and executor and
parallel-worker code to make this happen, hopefully not breaking too
many things along the way.
Examples with patch applied:
(1) non-parallel:
test=# explain analyze insert into primary_tbl select * from third_tbl;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------
Insert on primary_tbl (cost=0.00..154.99 rows=9999 width=12) (actual
time=108.445..108.446 rows=0 loops=1)
-> Seq Scan on third_tbl (cost=0.00..154.99 rows=9999 width=12)
(actual time=0.009..5.282 rows=9999 loops=1)
Planning Time: 0.132 ms
Execution Time: 108.596 ms
(4 rows)
(2) parallel:
test=# explain analyze insert into primary_tbl select * from third_tbl;
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------------
Gather (cost=0.00..16.00 rows=9999 width=12) (actual
time=69.870..70.310 rows=0 loops=1)
Workers Planned: 5
Workers Launched: 5
-> Parallel Insert on primary_tbl (cost=0.00..16.00 rows=500
width=12) (actual time=59.948..59.949 rows=0 loops=6)
-> Parallel Seq Scan on third_tbl (cost=0.00..80.00
rows=2500 width=12) (actual time=0.014..0.922 rows=1666 loops=6)
Planning Time: 0.121 ms
Execution Time: 70.438 ms
(7 rows)
(3) parallel select only (insert into table with foreign key)
test=# explain analyze insert into secondary_tbl select * from third_tbl;
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------------
Insert on secondary_tbl (cost=0.00..80.00 rows=9999 width=12)
(actual time=33.864..33.926 rows=0 loops=1)
-> Gather (cost=0.00..80.00 rows=9999 width=12) (actual
time=0.451..5.201 rows=9999 loops=1)
Workers Planned: 4
Workers Launched: 4
-> Parallel Seq Scan on third_tbl (cost=0.00..80.00
rows=2500 width=12) (actual time=0.013..0.717 rows=2000 loops=5)
Planning Time: 0.127 ms
Trigger for constraint secondary_tbl_index_fkey: time=331.834 calls=9999
Execution Time: 367.342 ms
(8 rows)
Known issues/TODOs:
- Currently only for "INSERT INTO ... SELECT ...". To support "INSERT
INTO ... VALUES ..." would need additional Table AM functions for
dividing up the INSERT work amongst the workers (currently only exists
for scans).
- When INSERTs are made parallel, currently the reported row-count in
the "INSERT 0 <row-count>" status only reflects the rows that the
leader has processed (not the workers) - so it is obviously less than
the actual number of rows inserted.
- Functions relating to computing the number of parallel workers for
an INSERT, and the cost of an INSERT, need work.
- "force_parallel_mode" handling was updated so that it only affects
SELECT (not INSERT) - can't allow it for INSERT because we're only
supporting "INSERT INTO .. SELECT ..." and don't support other types
of INSERTs, and also can't allow attempted parallel UPDATEs resulting
from "INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE" etc.
Thoughts and feedback?
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
0001-ParallelInsertSelect.patchapplication/octet-stream; name=0001-ParallelInsertSelect.patchDownload
diff --git a/src/backend/access/common/toast_internals.c b/src/backend/access/common/toast_internals.c
index 25a81e5..3e3aded 100644
--- a/src/backend/access/common/toast_internals.c
+++ b/src/backend/access/common/toast_internals.c
@@ -17,6 +17,7 @@
#include "access/genam.h"
#include "access/heapam.h"
#include "access/heaptoast.h"
+#include "access/parallel.h"
#include "access/table.h"
#include "access/toast_internals.h"
#include "access/xact.h"
@@ -116,7 +117,7 @@ toast_save_datum(Relation rel, Datum value,
TupleDesc toasttupDesc;
Datum t_values[3];
bool t_isnull[3];
- CommandId mycid = GetCurrentCommandId(true);
+ CommandId mycid = GetCurrentCommandId(!IsParallelWorker());
struct varlena *result;
struct varatt_external toast_pointer;
union
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..94c8507 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
-
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
tup->t_data->t_infomask |= HEAP_XMAX_INVALID;
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index dcaea71..f661e8c 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -822,19 +822,14 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
isdead = false;
break;
case HEAPTUPLE_INSERT_IN_PROGRESS:
-
/*
* Since we hold exclusive lock on the relation, normally the
* only way to see this is if it was inserted earlier in our
* own transaction. However, it can happen in system
* catalogs, since we tend to release write lock before commit
- * there. Give a warning if neither case applies; but in any
- * case we had better copy it.
+ * there. In any case we had better copy it.
*/
- if (!is_system_catalog &&
- !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data)))
- elog(WARNING, "concurrent insert in progress within table \"%s\"",
- RelationGetRelationName(OldHeap));
+
/* treat as live */
isdead = false;
break;
@@ -1434,16 +1429,11 @@ heapam_index_build_range_scan(Relation heapRelation,
* the only way to see this is if it was inserted earlier
* in our own transaction. However, it can happen in
* system catalogs, since we tend to release write lock
- * before commit there. Give a warning if neither case
- * applies.
+ * before commit there.
*/
xwait = HeapTupleHeaderGetXmin(heapTuple->t_data);
if (!TransactionIdIsCurrentTransactionId(xwait))
{
- if (!is_system_catalog)
- elog(WARNING, "concurrent insert in progress within table \"%s\"",
- RelationGetRelationName(heapRelation));
-
/*
* If we are performing uniqueness checks, indexing
* such a tuple could lead to a bogus uniqueness
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index a4944fa..9d3f100 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -53,13 +53,6 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs after that point.
- */
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
-
- /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..1442ec0 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,37 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * AssignFullTransactionIdForWorker
+ *
+ * For parallel operations, transaction id of leader will be used by the workers.
+ */
+void
+AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert((IsInParallelMode() || IsParallelWorker()));
+ s->fullTransactionId = fullTransactionId;
+}
+
+/*
+ * AssignCommandIdForWorker
+ *
+ * For parallel operations, command id of leader will be used by the workers.
+ */
+void
+AssignCommandIdForWorker(CommandId commandId, bool used)
+{
+ Assert((IsInParallelMode() || IsParallelWorker()));
+
+ /* this is global to a transaction, not subtransaction-local */
+ if (used)
+ currentCommandIdUsed = true;
+
+ currentCommandId = commandId;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -577,13 +608,6 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs at this point.
- */
- if (IsInParallelMode() || IsParallelWorker())
- elog(ERROR, "cannot assign XIDs during a parallel operation");
-
- /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Mustn't recurse here, or we might get a stack
* overflow if we're at the bottom of a huge stack of subtransactions none
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2e27e26..3fac2c2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -173,7 +173,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* against performing unsafe operations in parallel mode, but this gives a
* more user-friendly error message.
*/
- if ((XactReadOnly || IsInParallelMode()) &&
+ if ((XactReadOnly || (IsInParallelMode() && queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
@@ -235,7 +235,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
case CMD_INSERT:
case CMD_DELETE:
case CMD_UPDATE:
- estate->es_output_cid = GetCurrentCommandId(true);
+ estate->es_output_cid = GetCurrentCommandId(!IsParallelWorker());
break;
default:
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 382e78f..88eb286 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -77,6 +78,8 @@ typedef struct FixedParallelExecutorState
dsa_pointer param_exec;
int eflags;
int jit_flags;
+ FullTransactionId full_transaction_id;
+ CommandId command_id;
} FixedParallelExecutorState;
/*
@@ -173,18 +176,20 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ estate->es_plannedstmt->commandType == CMD_INSERT);
+ pstmt->commandType = (plan->type == T_ModifyTable) ? CMD_INSERT : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
- pstmt->hasModifyingCTE = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
+ pstmt->hasModifyingCTE = estate->es_plannedstmt->hasModifyingCTE;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
pstmt->dependsOnRole = false;
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
- pstmt->rootResultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
+ pstmt->rootResultRelations = estate->es_plannedstmt->rootResultRelations;
pstmt->appendRelations = NIL;
/*
@@ -730,6 +735,16 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
fpes->param_exec = InvalidDsaPointer;
fpes->eflags = estate->es_top_eflags;
fpes->jit_flags = estate->es_jit_flags;
+ if (pcxt->nworkers > 0 && planstate->type == T_ModifyTableState)
+ {
+ fpes->full_transaction_id = GetCurrentFullTransactionId();
+ fpes->command_id = GetCurrentCommandId(true);
+ }
+ else
+ {
+ fpes->full_transaction_id = InvalidFullTransactionId;
+ fpes->command_id = InvalidCommandId;
+ }
shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXECUTOR_FIXED, fpes);
/* Store query string */
@@ -1392,6 +1407,15 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
/* Get fixed-size state. */
fpes = shm_toc_lookup(toc, PARALLEL_KEY_EXECUTOR_FIXED, false);
+ if (FullTransactionIdIsValid(fpes->full_transaction_id))
+ {
+ AssignFullTransactionIdForWorker(fpes->full_transaction_id);
+ }
+ if (fpes->command_id != InvalidCommandId)
+ {
+ AssignCommandIdForWorker(fpes->command_id, true);
+ }
+
/* Set up DestReceiver, SharedExecutorInstrumentation, and QueryDesc. */
receiver = ExecParallelGetReceiver(seg, toc);
instrumentation = shm_toc_lookup(toc, PARALLEL_KEY_INSTRUMENTATION, true);
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..b107944 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -60,6 +60,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +105,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (outerNode->type == T_ModifyTable && ((ModifyTable *)outerNode)->returningLists != NULL) ?
+ ((ModifyTableState *)outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b399592..7e90df5 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -3903,6 +3903,36 @@ compute_parallel_worker(RelOptInfo *rel, double heap_pages, double index_pages,
}
/*
+ * Compute the number of parallel workers that should be used to insert into
+ * a relation.
+ *
+ * "max_workers" is caller's limit on the number of workers. This typically
+ * comes from a GUC.
+ */
+int
+compute_parallel_insert_worker(RelOptInfo *rel, int max_workers)
+{
+ int parallel_workers = 0;
+
+ /*
+ * If the user has set the parallel_workers reloption, use that; otherwise
+ * select a default number of workers.
+ */
+ if (rel->rel_parallel_workers != -1)
+ parallel_workers = rel->rel_parallel_workers;
+ else
+ {
+ /* TODO - smarts for computing best no. of workers for insert */
+ parallel_workers = max_workers;
+ }
+
+ /* In no case use more than caller supplied maximum number of workers */
+ parallel_workers = Min(parallel_workers, max_workers);
+
+ return parallel_workers;
+}
+
+/*
* generate_partitionwise_join_paths
* Create paths representing partitionwise join for given partitioned
* join relation.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index cd3716d..2a6c27a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -179,6 +179,7 @@ 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);
static double get_parallel_divisor(Path *path);
+static double get_modifytable_parallel_divisor(ModifyTablePath *path);
/*
@@ -202,6 +203,62 @@ clamp_row_est(double nrows)
}
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ double total_rows;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ total_rows = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ total_rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+
+ /* Adjust costing for parallelism, if used. */
+ if (path->path.parallel_workers > 0)
+ {
+ double parallel_divisor = get_modifytable_parallel_divisor(path);
+
+ /* The total cost is divided among all the workers. */
+ path->path.total_cost /= parallel_divisor;
+
+ /*
+ * In the case of a parallel plan, the row count needs to represent
+ * the number of tuples processed per worker.
+ */
+ path->path.rows = clamp_row_est(total_rows / parallel_divisor);
+ }
+ else
+ {
+ path->path.rows = total_rows;
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should report zero if no RETURNING, else an average
+ * of the RETURNING tlist widths. But it's what happened historically,
+ * and improving it is a task for another day.
+ */
+ if (total_rows > 0)
+ total_size /= total_rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
@@ -5737,6 +5794,29 @@ get_parallel_divisor(Path *path)
}
/*
+ * Divisor for ModifyTable (currently only Parallel Insert).
+ * Estimate the fraction of the work that each worker will do given the
+ * number of workers budgeted for the path.
+ * TODO: Needs revising based on further experience.
+ */
+static double
+get_modifytable_parallel_divisor(ModifyTablePath *path)
+{
+ double parallel_divisor = path->path.parallel_workers;
+
+ if (parallel_leader_participation && path->returningLists != NIL)
+ {
+ double leader_contribution;
+
+ leader_contribution = 1.0 - (0.3 * path->path.parallel_workers);
+ if (leader_contribution > 0)
+ parallel_divisor += leader_contribution;
+ }
+
+ return parallel_divisor;
+}
+
+/*
* compute_bitmap_pages
*
* compute number of pages fetched from heap in bitmap heap scan.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99278ee..cb9a6ea25 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(plan->lefttree, ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 8007e20..982db80 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
/* Determine what fraction of the plan is likely to be scanned */
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
@@ -1798,7 +1799,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1846,6 +1848,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_insert_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2382,13 +2385,97 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider Parallel INSERT */
+ if (parse->commandType == CMD_INSERT &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelInsertWorkers;
+
+ parallelInsertWorkers = compute_parallel_insert_worker(current_rel, max_parallel_workers_per_gather);
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelInsertWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_insert_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2405,6 +2492,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_insert_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7574,7 +7667,43 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (root->parse->commandType == CMD_INSERT)
+ {
+ Relation relation;
+ bool hasFKs;
+ RangeTblEntry *rte;
+ List *cachedFKs;
+
+ /* Check if the target relation has foreign keys; if so, avoid
+ * creating a parallel Insert plan (because inserting into
+ * such tables would result in creation of new CommandIds, and
+ * this isn't supported by parallel workers).
+ * Similarly, avoid creating a parallel Insert plan if ON
+ * CONFLICT ... DO UPDATE ... has been specified, because
+ * parallel UPDATE is not supported.
+ * However, do allow any underlying query to be run by parallel
+ * workers in these cases.
+ */
+
+ rte = rt_fetch(root->parse->resultRelation, root->parse->rtable);
+ relation = table_open(rte->relid, NoLock);
+ cachedFKs = RelationGetFKeyList(relation);
+ hasFKs = cachedFKs != NIL;
+ table_close(relation, NoLock);
+
+ if (hasFKs || (root->parse->onConflict != NULL && root->parse->onConflict->action == ONCONFLICT_UPDATE))
+ {
+ generate_useful_gather_paths(root, rel, false);
+ /* Don't allow parallel insert */
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index baefe0e..67a7cc8 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -225,6 +225,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -275,7 +276,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan->type == T_Gather || finalPlan->type == T_GatherMerge)
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (subplan->type == T_ModifyTable && ((ModifyTable *)subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = outerPlan(finalPlan)->targetlist;
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index c1fc866..4a9c3fa 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3538,11 +3538,11 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallel_workers)
{
+ ListCell *lc;
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
- ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
Assert(list_length(resultRelations) == list_length(subroots));
@@ -3557,45 +3557,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->path.pathtarget = rel->reltarget;
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
- pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
+ pathnode->path.parallel_safe = rel->consider_parallel;
+ if (rel->consider_parallel)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ foreach (lc, subpaths)
+ {
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
+ }
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallel_workers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3611,6 +3588,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index df1b43a..71a6c9b 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -385,6 +385,8 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId);
+extern void AssignCommandIdForWorker(CommandId commandId, bool used);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 10b6e81..1a01dcf 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -58,6 +58,8 @@ extern void generate_useful_gather_paths(PlannerInfo *root, RelOptInfo *rel,
bool override_rows);
extern int compute_parallel_worker(RelOptInfo *rel, double heap_pages,
double index_pages, int max_workers);
+extern int compute_parallel_insert_worker(RelOptInfo *rel,
+ int max_workers);
extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
Path *bitmapqual);
extern void generate_partitionwise_join_paths(PlannerInfo *root,
- When INSERTs are made parallel, currently the reported row-count in
the "INSERT 0 <row-count>" status only reflects the rows that the
leader has processed (not the workers) - so it is obviously less than
the actual number of rows inserted.
Attached an updated patch which fixes this issue (for parallel
INSERTs, each worker's processed tuple count is communicated in shared
memory back to the leader, where it is added to the global
"es_processed" count).
Attachments:
0002-ParallelInsertSelect.patchapplication/octet-stream; name=0002-ParallelInsertSelect.patchDownload
diff --git a/src/backend/access/common/toast_internals.c b/src/backend/access/common/toast_internals.c
index 25a81e5..3e3aded 100644
--- a/src/backend/access/common/toast_internals.c
+++ b/src/backend/access/common/toast_internals.c
@@ -17,6 +17,7 @@
#include "access/genam.h"
#include "access/heapam.h"
#include "access/heaptoast.h"
+#include "access/parallel.h"
#include "access/table.h"
#include "access/toast_internals.h"
#include "access/xact.h"
@@ -116,7 +117,7 @@ toast_save_datum(Relation rel, Datum value,
TupleDesc toasttupDesc;
Datum t_values[3];
bool t_isnull[3];
- CommandId mycid = GetCurrentCommandId(true);
+ CommandId mycid = GetCurrentCommandId(!IsParallelWorker());
struct varlena *result;
struct varatt_external toast_pointer;
union
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..94c8507 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
-
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
tup->t_data->t_infomask |= HEAP_XMAX_INVALID;
diff --git a/src/backend/access/heap/heapam_handler.c b/src/backend/access/heap/heapam_handler.c
index dcaea71..f661e8c 100644
--- a/src/backend/access/heap/heapam_handler.c
+++ b/src/backend/access/heap/heapam_handler.c
@@ -822,19 +822,14 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
isdead = false;
break;
case HEAPTUPLE_INSERT_IN_PROGRESS:
-
/*
* Since we hold exclusive lock on the relation, normally the
* only way to see this is if it was inserted earlier in our
* own transaction. However, it can happen in system
* catalogs, since we tend to release write lock before commit
- * there. Give a warning if neither case applies; but in any
- * case we had better copy it.
+ * there. In any case we had better copy it.
*/
- if (!is_system_catalog &&
- !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data)))
- elog(WARNING, "concurrent insert in progress within table \"%s\"",
- RelationGetRelationName(OldHeap));
+
/* treat as live */
isdead = false;
break;
@@ -1434,16 +1429,11 @@ heapam_index_build_range_scan(Relation heapRelation,
* the only way to see this is if it was inserted earlier
* in our own transaction. However, it can happen in
* system catalogs, since we tend to release write lock
- * before commit there. Give a warning if neither case
- * applies.
+ * before commit there.
*/
xwait = HeapTupleHeaderGetXmin(heapTuple->t_data);
if (!TransactionIdIsCurrentTransactionId(xwait))
{
- if (!is_system_catalog)
- elog(WARNING, "concurrent insert in progress within table \"%s\"",
- RelationGetRelationName(heapRelation));
-
/*
* If we are performing uniqueness checks, indexing
* such a tuple could lead to a bogus uniqueness
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index a4944fa..9d3f100 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -53,13 +53,6 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs after that point.
- */
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
-
- /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..1442ec0 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,37 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * AssignFullTransactionIdForWorker
+ *
+ * For parallel operations, transaction id of leader will be used by the workers.
+ */
+void
+AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert((IsInParallelMode() || IsParallelWorker()));
+ s->fullTransactionId = fullTransactionId;
+}
+
+/*
+ * AssignCommandIdForWorker
+ *
+ * For parallel operations, command id of leader will be used by the workers.
+ */
+void
+AssignCommandIdForWorker(CommandId commandId, bool used)
+{
+ Assert((IsInParallelMode() || IsParallelWorker()));
+
+ /* this is global to a transaction, not subtransaction-local */
+ if (used)
+ currentCommandIdUsed = true;
+
+ currentCommandId = commandId;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -577,13 +608,6 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs at this point.
- */
- if (IsInParallelMode() || IsParallelWorker())
- elog(ERROR, "cannot assign XIDs during a parallel operation");
-
- /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Mustn't recurse here, or we might get a stack
* overflow if we're at the bottom of a huge stack of subtransactions none
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2e27e26..3fac2c2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -173,7 +173,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* against performing unsafe operations in parallel mode, but this gives a
* more user-friendly error message.
*/
- if ((XactReadOnly || IsInParallelMode()) &&
+ if ((XactReadOnly || (IsInParallelMode() && queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
@@ -235,7 +235,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
case CMD_INSERT:
case CMD_DELETE:
case CMD_UPDATE:
- estate->es_output_cid = GetCurrentCommandId(true);
+ estate->es_output_cid = GetCurrentCommandId(!IsParallelWorker());
break;
default:
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 382e78f..9b05bff 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -77,6 +79,8 @@ typedef struct FixedParallelExecutorState
dsa_pointer param_exec;
int eflags;
int jit_flags;
+ FullTransactionId full_transaction_id;
+ CommandId command_id;
} FixedParallelExecutorState;
/*
@@ -173,18 +177,20 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ estate->es_plannedstmt->commandType == CMD_INSERT);
+ pstmt->commandType = (plan->type == T_ModifyTable) ? CMD_INSERT : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
- pstmt->hasModifyingCTE = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
+ pstmt->hasModifyingCTE = estate->es_plannedstmt->hasModifyingCTE;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
pstmt->dependsOnRole = false;
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
- pstmt->rootResultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
+ pstmt->rootResultRelations = estate->es_plannedstmt->rootResultRelations;
pstmt->appendRelations = NIL;
/*
@@ -591,6 +597,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -676,6 +683,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (planstate->plan->type == T_ModifyTable)
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -730,6 +745,16 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
fpes->param_exec = InvalidDsaPointer;
fpes->eflags = estate->es_top_eflags;
fpes->jit_flags = estate->es_jit_flags;
+ if (pcxt->nworkers > 0 && planstate->type == T_ModifyTableState)
+ {
+ fpes->full_transaction_id = GetCurrentFullTransactionId();
+ fpes->command_id = GetCurrentCommandId(true);
+ }
+ else
+ {
+ fpes->full_transaction_id = InvalidFullTransactionId;
+ fpes->command_id = InvalidCommandId;
+ }
shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXECUTOR_FIXED, fpes);
/* Store query string */
@@ -765,6 +790,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (planstate->plan->type == T_ModifyTable)
+ {
+ /* Allocate space for each worker's returned "# of tuples processed" count. */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1153,6 +1191,16 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ * This is currently done only in the case of parallel INSERT.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1380,6 +1428,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1392,6 +1441,15 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
/* Get fixed-size state. */
fpes = shm_toc_lookup(toc, PARALLEL_KEY_EXECUTOR_FIXED, false);
+ if (FullTransactionIdIsValid(fpes->full_transaction_id))
+ {
+ AssignFullTransactionIdForWorker(fpes->full_transaction_id);
+ }
+ if (fpes->command_id != InvalidCommandId)
+ {
+ AssignCommandIdForWorker(fpes->command_id, true);
+ }
+
/* Set up DestReceiver, SharedExecutorInstrumentation, and QueryDesc. */
receiver = ExecParallelGetReceiver(seg, toc);
instrumentation = shm_toc_lookup(toc, PARALLEL_KEY_INSTRUMENTATION, true);
@@ -1459,6 +1517,13 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (queryDesc->operation == CMD_INSERT)
+ {
+ /* Report the # of tuples processed during parallel INSERT execution. */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..b107944 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -60,6 +60,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +105,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (outerNode->type == T_ModifyTable && ((ModifyTable *)outerNode)->returningLists != NULL) ?
+ ((ModifyTableState *)outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b399592..7e90df5 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -3903,6 +3903,36 @@ compute_parallel_worker(RelOptInfo *rel, double heap_pages, double index_pages,
}
/*
+ * Compute the number of parallel workers that should be used to insert into
+ * a relation.
+ *
+ * "max_workers" is caller's limit on the number of workers. This typically
+ * comes from a GUC.
+ */
+int
+compute_parallel_insert_worker(RelOptInfo *rel, int max_workers)
+{
+ int parallel_workers = 0;
+
+ /*
+ * If the user has set the parallel_workers reloption, use that; otherwise
+ * select a default number of workers.
+ */
+ if (rel->rel_parallel_workers != -1)
+ parallel_workers = rel->rel_parallel_workers;
+ else
+ {
+ /* TODO - smarts for computing best no. of workers for insert */
+ parallel_workers = max_workers;
+ }
+
+ /* In no case use more than caller supplied maximum number of workers */
+ parallel_workers = Min(parallel_workers, max_workers);
+
+ return parallel_workers;
+}
+
+/*
* generate_partitionwise_join_paths
* Create paths representing partitionwise join for given partitioned
* join relation.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index cd3716d..f34b4c5 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -179,6 +179,7 @@ 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);
static double get_parallel_divisor(Path *path);
+static double get_modifytable_parallel_divisor(ModifyTablePath *path);
/*
@@ -203,6 +204,66 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ double total_rows;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ total_rows = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ total_rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+
+ /* Adjust costing for parallelism, if used. */
+ if (path->path.parallel_workers > 0)
+ {
+ double parallel_divisor = get_modifytable_parallel_divisor(path);
+
+ /* The total cost is divided among all the workers. */
+ path->path.total_cost /= parallel_divisor;
+
+ /*
+ * In the case of a parallel plan, the row count needs to represent
+ * the number of tuples processed per worker.
+ */
+ path->path.rows = clamp_row_est(total_rows / parallel_divisor);
+ }
+ else
+ {
+ path->path.rows = total_rows;
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should report zero if no RETURNING, else an average
+ * of the RETURNING tlist widths. But it's what happened historically,
+ * and improving it is a task for another day.
+ */
+ if (total_rows > 0)
+ total_size /= total_rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
@@ -5737,6 +5798,29 @@ get_parallel_divisor(Path *path)
}
/*
+ * Divisor for ModifyTable (currently only Parallel Insert).
+ * Estimate the fraction of the work that each worker will do given the
+ * number of workers budgeted for the path.
+ * TODO: Needs revising based on further experience.
+ */
+static double
+get_modifytable_parallel_divisor(ModifyTablePath *path)
+{
+ double parallel_divisor = path->path.parallel_workers;
+
+ if (parallel_leader_participation && path->returningLists != NIL)
+ {
+ double leader_contribution;
+
+ leader_contribution = 1.0 - (0.3 * path->path.parallel_workers);
+ if (leader_contribution > 0)
+ parallel_divisor += leader_contribution;
+ }
+
+ return parallel_divisor;
+}
+
+/*
* compute_bitmap_pages
*
* compute number of pages fetched from heap in bitmap heap scan.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99278ee..cb9a6ea25 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(plan->lefttree, ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 8007e20..982db80 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
/* Determine what fraction of the plan is likely to be scanned */
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
@@ -1798,7 +1799,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1846,6 +1848,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_insert_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2382,13 +2385,97 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider Parallel INSERT */
+ if (parse->commandType == CMD_INSERT &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelInsertWorkers;
+
+ parallelInsertWorkers = compute_parallel_insert_worker(current_rel, max_parallel_workers_per_gather);
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelInsertWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_insert_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2405,6 +2492,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_insert_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7574,7 +7667,43 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (root->parse->commandType == CMD_INSERT)
+ {
+ Relation relation;
+ bool hasFKs;
+ RangeTblEntry *rte;
+ List *cachedFKs;
+
+ /* Check if the target relation has foreign keys; if so, avoid
+ * creating a parallel Insert plan (because inserting into
+ * such tables would result in creation of new CommandIds, and
+ * this isn't supported by parallel workers).
+ * Similarly, avoid creating a parallel Insert plan if ON
+ * CONFLICT ... DO UPDATE ... has been specified, because
+ * parallel UPDATE is not supported.
+ * However, do allow any underlying query to be run by parallel
+ * workers in these cases.
+ */
+
+ rte = rt_fetch(root->parse->resultRelation, root->parse->rtable);
+ relation = table_open(rte->relid, NoLock);
+ cachedFKs = RelationGetFKeyList(relation);
+ hasFKs = cachedFKs != NIL;
+ table_close(relation, NoLock);
+
+ if (hasFKs || (root->parse->onConflict != NULL && root->parse->onConflict->action == ONCONFLICT_UPDATE))
+ {
+ generate_useful_gather_paths(root, rel, false);
+ /* Don't allow parallel insert */
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index baefe0e..67a7cc8 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -225,6 +225,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -275,7 +276,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan->type == T_Gather || finalPlan->type == T_GatherMerge)
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (subplan->type == T_ModifyTable && ((ModifyTable *)subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = outerPlan(finalPlan)->targetlist;
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index c1fc866..4a9c3fa 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3538,11 +3538,11 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallel_workers)
{
+ ListCell *lc;
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
- ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
Assert(list_length(resultRelations) == list_length(subroots));
@@ -3557,45 +3557,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->path.pathtarget = rel->reltarget;
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
- pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
+ pathnode->path.parallel_safe = rel->consider_parallel;
+ if (rel->consider_parallel)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ foreach (lc, subpaths)
+ {
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
+ }
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallel_workers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3611,6 +3588,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index df1b43a..71a6c9b 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -385,6 +385,8 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId);
+extern void AssignCommandIdForWorker(CommandId commandId, bool used);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..afb8a57 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 10b6e81..1a01dcf 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -58,6 +58,8 @@ extern void generate_useful_gather_paths(PlannerInfo *root, RelOptInfo *rel,
bool override_rows);
extern int compute_parallel_worker(RelOptInfo *rel, double heap_pages,
double index_pages, int max_workers);
+extern int compute_parallel_insert_worker(RelOptInfo *rel,
+ int max_workers);
extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
Path *bitmapqual);
extern void generate_partitionwise_join_paths(PlannerInfo *root,
On Tue, Sep 22, 2020 at 10:26 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Hi Hackers,
Following on from Dilip Kumar's POC patch for allowing parallelism of
the SELECT part of "INSERT INTO ... SELECT ...", I have attached a POC
patch for allowing parallelism of both the INSERT and SELECT parts,
where it can be allowed.
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.
Obviously I've had to update the planner and executor and
parallel-worker code to make this happen, hopefully not breaking too
many things along the way.
I feel this will be a very good performance improvement. +1 for this.
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
Hi,
On 2020-09-22 14:55:21 +1000, Greg Nancarrow wrote:
Following on from Dilip Kumar's POC patch for allowing parallelism of
the SELECT part of "INSERT INTO ... SELECT ...", I have attached a POC
patch for allowing parallelism of both the INSERT and SELECT parts,
where it can be allowed.
Cool!
I think it'd be good if you outlined what your approach is to make this
safe.
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.
I think it'd be good to do this part separately and first, independent
of whether the insert part can be parallelized.
Obviously I've had to update the planner and executor and
parallel-worker code to make this happen, hopefully not breaking too
many things along the way.
Hm, it looks like you've removed a fair bit of checks, it's not clear to
me why that's safe in each instance.
- Currently only for "INSERT INTO ... SELECT ...". To support "INSERT
INTO ... VALUES ..." would need additional Table AM functions for
dividing up the INSERT work amongst the workers (currently only exists
for scans).
Hm, not entirely following. What precisely are you thinking of here?
I doubt it's really worth adding parallelism support for INSERT
... VALUES, the cost of spawning workers will almost always higher than
the benefit.
@@ -116,7 +117,7 @@ toast_save_datum(Relation rel, Datum value, TupleDesc toasttupDesc; Datum t_values[3]; bool t_isnull[3]; - CommandId mycid = GetCurrentCommandId(true); + CommandId mycid = GetCurrentCommandId(!IsParallelWorker()); struct varlena *result; struct varatt_external toast_pointer; union
Hm? Why do we need this in the various places you have made this change?
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c index 1585861..94c8507 100644 --- a/src/backend/access/heap/heapam.c +++ b/src/backend/access/heap/heapam.c @@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid, * inserts in general except for the cases where inserts generate a new * CommandId (eg. inserts into a table having a foreign key column). */ - if (IsParallelWorker()) - ereport(ERROR, - (errcode(ERRCODE_INVALID_TRANSACTION_STATE), - errmsg("cannot insert tuples in a parallel worker"))); -
I'm afraid that this weakens our checks more than I'd like. What if this
ends up being invoked from inside C code?
@@ -822,19 +822,14 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap, isdead = false; break; case HEAPTUPLE_INSERT_IN_PROGRESS: - /* * Since we hold exclusive lock on the relation, normally the * only way to see this is if it was inserted earlier in our * own transaction. However, it can happen in system * catalogs, since we tend to release write lock before commit - * there. Give a warning if neither case applies; but in any - * case we had better copy it. + * there. In any case we had better copy it. */ - if (!is_system_catalog && - !TransactionIdIsCurrentTransactionId(HeapTupleHeaderGetXmin(tuple->t_data))) - elog(WARNING, "concurrent insert in progress within table \"%s\"", - RelationGetRelationName(OldHeap)); + /* treat as live */ isdead = false; break; @@ -1434,16 +1429,11 @@ heapam_index_build_range_scan(Relation heapRelation, * the only way to see this is if it was inserted earlier * in our own transaction. However, it can happen in * system catalogs, since we tend to release write lock - * before commit there. Give a warning if neither case - * applies. + * before commit there. */ xwait = HeapTupleHeaderGetXmin(heapTuple->t_data); if (!TransactionIdIsCurrentTransactionId(xwait)) { - if (!is_system_catalog) - elog(WARNING, "concurrent insert in progress within table \"%s\"", - RelationGetRelationName(heapRelation)); - /* * If we are performing uniqueness checks, indexing * such a tuple could lead to a bogus uniqueness
Huh, I don't think this should be necessary?
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c index a4944fa..9d3f100 100644 --- a/src/backend/access/transam/varsup.c +++ b/src/backend/access/transam/varsup.c @@ -53,13 +53,6 @@ GetNewTransactionId(bool isSubXact) TransactionId xid;/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs after that point.
- */
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
-
- /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
Same thing, this code cannot just be allowed to be reachable. What
prevents you from assigning two different xids from different workers
etc?
@@ -577,13 +608,6 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs at this point.
- */
- if (IsInParallelMode() || IsParallelWorker())
- elog(ERROR, "cannot assign XIDs during a parallel operation");
-
- /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Mustn't recurse here, or we might get a stack
* overflow if we're at the bottom of a huge stack of subtransactions none
Dito.
Greetings,
Andres Freund
On Tue, Sep 22, 2020 at 4:56 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Gather (cost=0.00..16.00 rows=9999 width=12) (actual
time=69.870..70.310 rows=0 loops=1)
Workers Planned: 5
Workers Launched: 5
-> Parallel Insert on primary_tbl (cost=0.00..16.00 rows=500
width=12) (actual time=59.948..59.949 rows=0 loops=6)
Nice. I took it for a quick spin. I was initially surprised to see
Gather. I suppose I thought that Parallel {Insert|Update|Delete}
might be a top level node itself, because in such a plan there is no
need to gather tuples per se. I understand exactly why you have it
that way though: Gather is needed to control workers and handle their
errors etc, and we don't want to have to terminate parallelism anyway
(thinking of some kind of plan with multiple write subqueries).
On Thu, Sep 24, 2020 at 7:57 AM Thomas Munro <thomas.munro@gmail.com> wrote:
On Tue, Sep 22, 2020 at 4:56 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Gather (cost=0.00..16.00 rows=9999 width=12) (actual
time=69.870..70.310 rows=0 loops=1)
Workers Planned: 5
Workers Launched: 5
-> Parallel Insert on primary_tbl (cost=0.00..16.00 rows=500
width=12) (actual time=59.948..59.949 rows=0 loops=6)Nice. I took it for a quick spin. I was initially surprised to see
Gather. I suppose I thought that Parallel {Insert|Update|Delete}
might be a top level node itself, because in such a plan there is no
need to gather tuples per se. I understand exactly why you have it
that way though: Gather is needed to control workers and handle their
errors etc, and we don't want to have to terminate parallelism anyway
(thinking of some kind of plan with multiple write subqueries).
I have not checked the patch but I guess if we parallelise Inserts
with Returning then isn't it better to have Gather node above Parallel
Inserts?
--
With Regards,
Amit Kapila.
On Thu, Sep 24, 2020 at 12:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I have not checked the patch but I guess if we parallelise Inserts
with Returning then isn't it better to have Gather node above Parallel
Inserts?
This is indeed the case with the patch applied.
For example:
test=# explain insert into primary_tbl select * from third_tbl
returning index, height;
QUERY PLAN
-----------------------------------------------------------------------------------
Gather (cost=0.00..28.15 rows=9999 width=12)
Workers Planned: 3
-> Parallel Insert on primary_tbl (cost=0.00..28.15 rows=1040 width=12)
-> Parallel Seq Scan on third_tbl (cost=0.00..87.25
rows=3225 width=12)
(4 rows)
test=# insert into primary_tbl select * from third_tbl returning index, height;
index | height
-------+--------
1 | 1.2
2 | 1.2
3 | 1.2
4 | 1.2
5 | 1.2
6 | 1.2
7 | 1.2
...
9435 | 1.2
9619 | 1.2
9620 | 1.2
(9999 rows)
INSERT 0 9999
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Sep 24, 2020 at 7:51 AM Andres Freund <andres@anarazel.de> wrote:
On 2020-09-22 14:55:21 +1000, Greg Nancarrow wrote:
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c index 1585861..94c8507 100644 --- a/src/backend/access/heap/heapam.c +++ b/src/backend/access/heap/heapam.c @@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid, * inserts in general except for the cases where inserts generate a new * CommandId (eg. inserts into a table having a foreign key column). */ - if (IsParallelWorker()) - ereport(ERROR, - (errcode(ERRCODE_INVALID_TRANSACTION_STATE), - errmsg("cannot insert tuples in a parallel worker"))); -I'm afraid that this weakens our checks more than I'd like.
I think we need to change/remove this check to allow inserts by
parallel workers. I am not sure but maybe we can add an Assert to
ensure that it is safe to perform insert via parallel worker.
What if this
ends up being invoked from inside C code?
I think it shouldn't be a problem unless one is trying to do something
like insert into foreign key table. So, probably we can have an Assert
to catch it if possible. Do you have any other idea?
--
With Regards,
Amit Kapila.
What if this
ends up being invoked from inside C code?I think it shouldn't be a problem unless one is trying to do something
like insert into foreign key table. So, probably we can have an Assert
to catch it if possible. Do you have any other idea?
Note that the planner code updated by the patch does avoid creating a
Parallel INSERT plan in the case of inserting into a table with a
foreign key (so commandIds won't be created in the parallel-worker
code).
I'm not sure how to distinguish the "invoked from inside C code" case though.
Regards,
Greg Nancarrow
Fujitsu Australia
Hi Andres,
On Thu, Sep 24, 2020 at 12:21 PM Andres Freund <andres@anarazel.de> wrote:
I think it'd be good if you outlined what your approach is to make this
safe.
Some prior work has already been done to establish the necessary
infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the
parallel-worker code (such as inserts into a table having a foreign
key) - these cases need to be avoided.
See the following commits.
85f6b49 Allow relation extension lock to conflict among parallel group members
3ba59cc Allow page lock to conflict among parallel group members
The planner code updated by the patch avoids creating a Parallel
INSERT plan in the case of inserting into a table that has a foreign
key.
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.I think it'd be good to do this part separately and first, independent
of whether the insert part can be parallelized.
OK then, I'll try to extract that as a separate patch.
Obviously I've had to update the planner and executor and
parallel-worker code to make this happen, hopefully not breaking too
many things along the way.Hm, it looks like you've removed a fair bit of checks, it's not clear to
me why that's safe in each instance.
It should be safe for Parallel INSERT - but you are right, these are
brute force removals (for the purpose of a POC patch) that should be
tightened up wherever possible to disallow unsafe paths into that
code. Problem is, currently there's not a lot of context information
available to easily allow that, so some work needs to be done.
- Currently only for "INSERT INTO ... SELECT ...". To support "INSERT
INTO ... VALUES ..." would need additional Table AM functions for
dividing up the INSERT work amongst the workers (currently only exists
for scans).Hm, not entirely following. What precisely are you thinking of here?
All I was saying is that for SELECTs, the work done by each parallel
worker is effectively divided up by parallel-worker-related functions
in tableam.c and indexam.c, and no such technology currently exists
for dividing up work for the "INSERT ... VALUES" case.
I doubt it's really worth adding parallelism support for INSERT
... VALUES, the cost of spawning workers will almost always higher than
the benefit.
You're probably right in doubting any benefit, but I wasn't entirely sure.
@@ -116,7 +117,7 @@ toast_save_datum(Relation rel, Datum value, TupleDesc toasttupDesc; Datum t_values[3]; bool t_isnull[3]; - CommandId mycid = GetCurrentCommandId(true); + CommandId mycid = GetCurrentCommandId(!IsParallelWorker()); struct varlena *result; struct varatt_external toast_pointer; unionHm? Why do we need this in the various places you have made this change?
It's because for Parallel INSERT, we're assigning the same command-id
to each worker up-front during worker initialization (the commandId
has been retrieved by the leader and passed through to each worker)
and "currentCommandIdUsed" has been set true. See the
AssignCommandIdForWorker() function in the patch.
If you see the code of GetCurrentCommandId(), you'll see it Assert
that it's not being run by a parallel worker if the parameter is true.
I didn't want to remove yet another check, without being able to know
the context of the caller, because only for Parallel INSERT do I know
that "currentCommandIdUsed was already true at the start of the
parallel operation". See the comment in that function. Anyway, that's
why I'm passing "false" to relevant GetCurrentCommandId() calls if
they're being run by a parallel (INSERT) worker.
@@ -822,19 +822,14 @@ heapam_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap, isdead = false; break; case HEAPTUPLE_INSERT_IN_PROGRESS: - /* * Since we hold exclusive lock on the relation, normally the * only way to see this is if it was inserted earlier in our * own transaction. However, it can happen in system * catalogs, since we tend to release write lock before >commit - * there. Give a warning if neither case applies; but in any - * case we had better copy it. + * there. In any case we had better copy it. */ - if (!is_system_catalog && - !TransactionIdIsCurrentTransactionId>(HeapTupleHeaderGetXmin(tuple->t_data))) - elog(WARNING, "concurrent insert in progress within >table \"%s\"", - RelationGetRelationName(OldHeap)); + /* treat as live */ isdead = false; break; @@ -1434,16 +1429,11 @@ heapam_index_build_range_scan(Relation heapRelation, * the only way to see this is if it was inserted >earlier * in our own transaction. However, it can happen in * system catalogs, since we tend to release write >lock - * before commit there. Give a warning if neither >case - * applies. + * before commit there. */ xwait = HeapTupleHeaderGetXmin(heapTuple->t_data); if (!TransactionIdIsCurrentTransactionId(xwait)) { - if (!is_system_catalog) - elog(WARNING, "concurrent insert in >progress within table \"%s\"", - RelationGetRelationName>(heapRelation)); - /* * If we are performing uniqueness checks, >>indexing * such a tuple could lead to a bogus >uniquenessHuh, I don't think this should be necessary?
Yes, I think you're right, I perhaps got carried away removing checks
on concurrent inserts. I will revert those changes.
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c index a4944fa..9d3f100 100644 --- a/src/backend/access/transam/varsup.c +++ b/src/backend/access/transam/varsup.c @@ -53,13 +53,6 @@ GetNewTransactionId(bool isSubXact) TransactionId xid;/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs after that point.
- */
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
-
- /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/Same thing, this code cannot just be allowed to be reachable. What
prevents you from assigning two different xids from different workers
etc?
At least in the case of Parallel INSERT, the leader for the Parallel
INSERT gets a new xid (GetCurrentFullTransactionId) and it is passed
through and assigned to each of the workers during their
initialization (so they are assigned the same xid).
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Sep 22, 2020 at 10:26 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.
Thanks Greg for the patch.
I have few points (inspired from parallel copy feature work) to mention:
1. What if the target table is a foreign table or partitioned table?
2. What happens if the target table has triggers(before statement,
after statement, before row, after row) that are parallel unsafe?
3. Will each worker be doing single row insertions or multi inserts?
If single row insertions, will the buffer lock contentions be more?
5. How does it behave with toast columns values?
6. How does it behave if we have a RETURNING clause with INSERT INTO SELECT?
I'm looking forward to seeing some initial numbers on execution times
with and without patch.
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Sep 25, 2020 at 7:01 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
I have few points (inspired from parallel copy feature work) to mention:
1. What if the target table is a foreign table or partitioned table?
2. What happens if the target table has triggers(before statement,
after statement, before row, after row) that are parallel unsafe?
3. Will each worker be doing single row insertions or multi inserts?
If single row insertions, will the buffer lock contentions be more?
5. How does it behave with toast columns values?
6. How does it behave if we have a RETURNING clause with INSERT INTO SELECT?
Hi Bharath,
Thanks for pointing out more cases I need to exclude and things I need
to investigate further.
I have taken note of them, and will do more testing and improvement.
At least RETURNING clause with INSERT INTO SELECT is working!
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Sep 25, 2020 at 10:02 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Hi Andres,
On Thu, Sep 24, 2020 at 12:21 PM Andres Freund <andres@anarazel.de> wrote:
@@ -116,7 +117,7 @@ toast_save_datum(Relation rel, Datum value, TupleDesc toasttupDesc; Datum t_values[3]; bool t_isnull[3]; - CommandId mycid = GetCurrentCommandId(true); + CommandId mycid = GetCurrentCommandId(!IsParallelWorker()); struct varlena *result; struct varatt_external toast_pointer; unionHm? Why do we need this in the various places you have made this change?
It's because for Parallel INSERT, we're assigning the same command-id
to each worker up-front during worker initialization (the commandId
has been retrieved by the leader and passed through to each worker)
and "currentCommandIdUsed" has been set true. See the
AssignCommandIdForWorker() function in the patch.
If you see the code of GetCurrentCommandId(), you'll see it Assert
that it's not being run by a parallel worker if the parameter is true.
I didn't want to remove yet another check, without being able to know
the context of the caller, because only for Parallel INSERT do I know
that "currentCommandIdUsed was already true at the start of the
parallel operation". See the comment in that function. Anyway, that's
why I'm passing "false" to relevant GetCurrentCommandId() calls if
they're being run by a parallel (INSERT) worker.
But we can tighten the condition in GetCurrentCommandId() such that it
Asserts for parallel worker only when currentCommandIdUsed is not set
before start of parallel operation. I also find these changes in the
callers of GetCurrentCommandId() quite adhoc and ugly even if they are
correct. Also, why we don't face a similar problems for parallel copy?
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c index a4944fa..9d3f100 100644 --- a/src/backend/access/transam/varsup.c +++ b/src/backend/access/transam/varsup.c @@ -53,13 +53,6 @@ GetNewTransactionId(bool isSubXact) TransactionId xid;/*
- * Workers synchronize transaction state at the beginning of each parallel
- * operation, so we can't account for new XIDs after that point.
- */
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
-
- /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/Same thing, this code cannot just be allowed to be reachable. What
prevents you from assigning two different xids from different workers
etc?At least in the case of Parallel INSERT, the leader for the Parallel
INSERT gets a new xid (GetCurrentFullTransactionId) and it is passed
through and assigned to each of the workers during their
initialization (so they are assigned the same xid).
So are you facing problems in this area because we EnterParallelMode
before even assigning the xid in the leader? Because I don't think we
should ever reach this code in the worker. If so, there are two
possibilities that come to my mind (a) assign xid in leader before
entering parallel mode or (b) change the check so that we don't assign
the new xid in workers. In this case, I am again wondering how does
parallel copy dealing this?
--
With Regards,
Amit Kapila.
On Fri, Sep 25, 2020 at 5:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
At least in the case of Parallel INSERT, the leader for the Parallel
INSERT gets a new xid (GetCurrentFullTransactionId) and it is passed
through and assigned to each of the workers during their
initialization (so they are assigned the same xid).So are you facing problems in this area because we EnterParallelMode
before even assigning the xid in the leader? Because I don't think we
should ever reach this code in the worker. If so, there are two
possibilities that come to my mind (a) assign xid in leader before
entering parallel mode or (b) change the check so that we don't assign
the new xid in workers. In this case, I am again wondering how does
parallel copy dealing this?
In parallel copy, we are doing option (a) i.e. the leader gets the
full txn id before entering parallel mode and passes it to all
workers.
In the leader:
full_transaction_id = GetCurrentFullTransactionId();
EnterParallelMode();
shared_info_ptr->full_transaction_id = full_transaction_id;
In the workers:
AssignFullTransactionIdForWorker(pcshared_info->full_transaction_id);
Hence below part of the code doesn't get hit.
if (IsInParallelMode() || IsParallelWorker())
elog(ERROR, "cannot assign XIDs during a parallel operation");
We also deal with the commandid similarly i.e. the leader gets the
command id, and workers would use it while insertion.
In the leader:
shared_info_ptr->mycid = GetCurrentCommandId(true);
In the workers:
AssignCommandIdForWorker(pcshared_info->mycid, true);
[1]: void AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId) { TransactionState s = CurrentTransactionState;
void
AssignFullTransactionIdForWorker(FullTransactionId fullTransactionId)
{
TransactionState s = CurrentTransactionState;
Assert((IsInParallelMode() || IsParallelWorker()));
s->fullTransactionId = fullTransactionId;
}
void
AssignCommandIdForWorker(CommandId commandId, bool used)
{
Assert((IsInParallelMode() || IsParallelWorker()));
/* this is global to a transaction, not subtransaction-local */
if (used)
currentCommandIdUsed = true;
currentCommandId = commandId;
}
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Sep 25, 2020 at 10:17 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
But we can tighten the condition in GetCurrentCommandId() such that it
Asserts for parallel worker only when currentCommandIdUsed is not set
before start of parallel operation. I also find these changes in the
callers of GetCurrentCommandId() quite adhoc and ugly even if they are
correct. Also, why we don't face a similar problems for parallel copy?
For Parallel Insert, as part of query plan execution,
GetCurrentCommandId(true) is being called as part of INSERT statement
execution.
Parallel Copy of course doesn't have to deal with this. That's why
there's a difference. And also, it has its own parallel entry point
(ParallelCopyMain), so it's in full control, it's not trying to fit in
with the infrastructure for plan execution.
So are you facing problems in this area because we EnterParallelMode
before even assigning the xid in the leader? Because I don't think we
should ever reach this code in the worker. If so, there are two
possibilities that come to my mind (a) assign xid in leader before
entering parallel mode or (b) change the check so that we don't assign
the new xid in workers. In this case, I am again wondering how does
parallel copy dealing this?
Again, there's a fundamental difference in the Parallel Insert case.
Right at the top of ExecutePlan it calls EnterParallelMode().
For ParallelCopy(), there is no such problem. EnterParallelMode() is
only called just before ParallelCopyMain() is called. So it can easily
acquire the xid before this, because then parallel mode is not set.
As it turns out, I think I have solved the commandId issue (and almost
the xid issue) by realising that both the xid and cid are ALREADY
being included as part of the serialized transaction state in the
Parallel DSM. So actually I don't believe that there is any need for
separately passing them in the DSM, and having to use those
AssignXXXXForWorker() functions in the worker code - not even in the
Parallel Copy case (? - need to check). GetCurrentCommandId(true) and
GetFullTransactionId() need to be called prior to Parallel DSM
initialization, so they are included in the serialized transaction
state.
I just needed to add a function to set currentCommandIdUsed=true in
the worker initialization (for INSERT case) and make a small tweak to
the Assert in GetCurrentCommandId() to ensure that
currentCommandIdUsed, in a parallel worker, never gets set to true
when it is false. This is in line with the comment in that function,
because we know that "currentCommandId was already true at the start
of the parallel operation". With this in place, I don't need to change
any of the original calls to GetCurrentCommandId(), so this addresses
that issue raised by Andres.
I am not sure yet how to get past the issue of the parallel mode being
set at the top of ExecutePlan(). With that in place, it doesn't allow
a xid to be acquired for the leader, without removing/changing that
parallel-mode check in GetNewTransactionId().
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Sep 25, 2020 at 9:23 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Sep 25, 2020 at 10:17 PM Amit Kapila <amit.kapila16@g
As it turns out, I think I have solved the commandId issue (and almost
the xid issue) by realising that both the xid and cid are ALREADY
being included as part of the serialized transaction state in the
Parallel DSM. So actually I don't believe that there is any need for
separately passing them in the DSM, and having to use those
AssignXXXXForWorker() functions in the worker code - not even in the
Parallel Copy case (? - need to check). GetCurrentCommandId(true) and
GetFullTransactionId() need to be called prior to Parallel DSM
initialization, so they are included in the serialized transaction
state.
I just needed to add a function to set currentCommandIdUsed=true in
the worker initialization (for INSERT case) and make a small tweak to
the Assert in GetCurrentCommandId() to ensure that
currentCommandIdUsed, in a parallel worker, never gets set to true
when it is false. This is in line with the comment in that function,
because we know that "currentCommandId was already true at the start
of the parallel operation". With this in place, I don't need to change
any of the original calls to GetCurrentCommandId(), so this addresses
that issue raised by Andres.I am not sure yet how to get past the issue of the parallel mode being
set at the top of ExecutePlan(). With that in place, it doesn't allow
a xid to be acquired for the leader, without removing/changing that
parallel-mode check in GetNewTransactionId().
I think now there is no fundamental problem in allocating xid in the
leader and then sharing it with workers who can use it to perform the
insert. So we can probably tweak that check so that it is true for
only parallel workers.
--
With Regards,
Amit Kapila.
On Fri, Sep 25, 2020 at 2:31 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
On Tue, Sep 22, 2020 at 10:26 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
For cases where it can't be allowed (e.g. INSERT into a table with
foreign keys, or INSERT INTO ... SELECT ... ON CONFLICT ... DO UPDATE
...") it at least allows parallelism of the SELECT part.Thanks Greg for the patch.
2. What happens if the target table has triggers(before statement,
after statement, before row, after row) that are parallel unsafe?
In such a case, the parallel insert shouldn't be selected. However, we
should still be able to execute the Select part in parallel.
3. Will each worker be doing single row insertions or multi inserts?
If single row insertions, will the buffer lock contentions be more?
I don't think the purpose of this patch is to change the basic flow of
how Insert works and also I am not sure if it is worth the effort as
well. I have answered this earlier in a bit more detailed way [1]/messages/by-id/CAA4eK1Ks8Sqs29VHPS6koNj5E9YQdkGCzgGsSrQMeUbQfe28yg@mail.gmail.com.
[1]: /messages/by-id/CAA4eK1Ks8Sqs29VHPS6koNj5E9YQdkGCzgGsSrQMeUbQfe28yg@mail.gmail.com
--
With Regards,
Amit Kapila.
On Fri, Sep 25, 2020 at 9:23 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Sep 25, 2020 at 10:17 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Again, there's a fundamental difference in the Parallel Insert case.
Right at the top of ExecutePlan it calls EnterParallelMode().
For ParallelCopy(), there is no such problem. EnterParallelMode() is
only called just before ParallelCopyMain() is called. So it can easily
acquire the xid before this, because then parallel mode is not set.As it turns out, I think I have solved the commandId issue (and almost
the xid issue) by realising that both the xid and cid are ALREADY
being included as part of the serialized transaction state in the
Parallel DSM. So actually I don't believe that there is any need for
separately passing them in the DSM, and having to use those
AssignXXXXForWorker() functions in the worker code - not even in the
Parallel Copy case (? - need to check).
Thanks Gred for the detailed points.
I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.
In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)
In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.
Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.
Thoughts?
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Sat, Sep 26, 2020 at 11:00 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
On Fri, Sep 25, 2020 at 9:23 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Sep 25, 2020 at 10:17 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Again, there's a fundamental difference in the Parallel Insert case.
Right at the top of ExecutePlan it calls EnterParallelMode().
For ParallelCopy(), there is no such problem. EnterParallelMode() is
only called just before ParallelCopyMain() is called. So it can easily
acquire the xid before this, because then parallel mode is not set.As it turns out, I think I have solved the commandId issue (and almost
the xid issue) by realising that both the xid and cid are ALREADY
being included as part of the serialized transaction state in the
Parallel DSM. So actually I don't believe that there is any need for
separately passing them in the DSM, and having to use those
AssignXXXXForWorker() functions in the worker code - not even in the
Parallel Copy case (? - need to check).Thanks Gred for the detailed points.
I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)
This won't be true for Parallel Insert patch as explained by Greg as
well because we enter-parallel-mode much before we assign xid.
--
With Regards,
Amit Kapila.
On Fri, Sep 25, 2020 at 9:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
What if this
ends up being invoked from inside C code?I think it shouldn't be a problem unless one is trying to do something
like insert into foreign key table. So, probably we can have an Assert
to catch it if possible. Do you have any other idea?Note that the planner code updated by the patch does avoid creating a
Parallel INSERT plan in the case of inserting into a table with a
foreign key (so commandIds won't be created in the parallel-worker
code).
I'm not sure how to distinguish the "invoked from inside C code" case though.
I think if possible we can have an Assert to check if it is a
parallel-worker and relation has a foreign-key. Similarly, we can
enhance the check for any other un-safe use. This will prevent the
illegal usage of inserts via parallel workers.
--
With Regards,
Amit Kapila.
On Wed, Sep 23, 2020 at 2:21 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
- When INSERTs are made parallel, currently the reported row-count in
the "INSERT 0 <row-count>" status only reflects the rows that the
leader has processed (not the workers) - so it is obviously less than
the actual number of rows inserted.Attached an updated patch which fixes this issue (for parallel
INSERTs, each worker's processed tuple count is communicated in shared
memory back to the leader, where it is added to the global
"es_processed" count).
I noticed that we are not having any check for skipping temporary
table insertion.
/* Check if the target relation has foreign keys; if so, avoid
* creating a parallel Insert plan (because inserting into
* such tables would result in creation of new CommandIds, and
* this isn't supported by parallel workers).
* Similarly, avoid creating a parallel Insert plan if ON
* CONFLICT ... DO UPDATE ... has been specified, because
* parallel UPDATE is not supported.
* However, do allow any underlying query to be run by parallel
* workers in these cases.
*/
You should also include temporary tables check here, as parallel
workers might not have access to temporary tables.
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
On Sun, Sep 27, 2020 at 2:03 AM vignesh C <vignesh21@gmail.com> wrote:
I noticed that we are not having any check for skipping temporary
table insertion.
You should also include temporary tables check here, as parallel
workers might not have access to temporary tables.
Thanks Vignesh, you are right, I need to test this and add it to the
list of further exclusions that the patch needs to check for.
Hopefully I can provide an updated patch soon that caters for these
additional identified cases.
Regards,
Greg Nancarrow
Fujitsu Australia
On Sat, Sep 26, 2020 at 3:30 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.
Hi Bharath,
I pretty much agree with your above points.
I've attached an updated Parallel INSERT...SELECT patch, that:
- Only uses existing transaction state serialization support for
transfer of xid and cid.
- Adds a "SetCurrentCommandIdUsedForWorker" function, for setting
currentCommandIdUsed=true at the start of a parallel operation (used
for Parallel INSERT case, where we know the currentCommandId has been
assigned to the worker at the start of the parallel operation).
- Tweaks the Assert condition within "used=true" parameter case in
GetCurrentCommandId(), so that it only fires if in a parallel worker
and currentCommandId is false - refer to the updated comment in that
function.
- Does not modify any existing GetCurrentCommandId() calls.
- Does not remove any existing parallel-related asserts/checks, except
for the "cannot insert tuples in a parallel worker" error in
heap_prepare_insert(). I am still considering what to do with the
original error-check here.
[- Does not yet cater for other exclusion cases that you and Vignesh
have pointed out]
This patch is mostly a lot cleaner, but does contain a possible ugly
hack, in that where it needs to call GetCurrentFullTransactionId(), it
must temporarily escape parallel-mode (recalling that parallel-mode is
set true right at the top of ExectePlan() in the cases of Parallel
INSERT/SELECT).
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
0003-ParallelInsertSelect.patchapplication/octet-stream; name=0003-ParallelInsertSelect.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..94c8507 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
-
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
tup->t_data->t_infomask |= HEAP_XMAX_INVALID;
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..8c69931 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -764,12 +778,13 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed
+ * if currentCommandIdUsed was already true at the start of the
+ * parallel operation (by way of SetCurrentCommandIdUsed()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision
+ * for communicating this back to the leader.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2e27e26..0f3bd82 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -173,7 +173,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* against performing unsafe operations in parallel mode, but this gives a
* more user-friendly error message.
*/
- if ((XactReadOnly || IsInParallelMode()) &&
+ if ((XactReadOnly || (IsInParallelMode() && queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 382e78f..2d6a200 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,18 +175,20 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ estate->es_plannedstmt->commandType == CMD_INSERT);
+ pstmt->commandType = (plan->type == T_ModifyTable) ? CMD_INSERT : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
- pstmt->hasModifyingCTE = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
+ pstmt->hasModifyingCTE = estate->es_plannedstmt->hasModifyingCTE;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
pstmt->dependsOnRole = false;
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
- pstmt->rootResultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
+ pstmt->rootResultRelations = estate->es_plannedstmt->rootResultRelations;
pstmt->appendRelations = NIL;
/*
@@ -591,6 +595,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -676,6 +681,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (planstate->plan->type == T_ModifyTable)
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -765,6 +778,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (planstate->plan->type == T_ModifyTable)
+ {
+ /* Allocate space for each worker's returned "# of tuples processed" count. */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1153,6 +1179,16 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ * This is currently done only in the case of parallel INSERT.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1380,6 +1416,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1401,6 +1438,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || queryDesc->operation == CMD_INSERT);
+ if (queryDesc->operation == CMD_INSERT)
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of
+ * the parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1459,6 +1506,13 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (queryDesc->operation == CMD_INSERT)
+ {
+ /* Report the # of tuples processed during parallel INSERT execution. */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..dd8a2db 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -60,6 +60,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +105,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (outerNode->type == T_ModifyTable && ((ModifyTable *)outerNode)->returningLists != NULL) ?
+ ((ModifyTableState *)outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -166,6 +169,29 @@ ExecGather(PlanState *pstate)
{
ParallelContext *pcxt;
+ /* For parallel INSERT, assign FullTransactionId and CurrentCommandId,
+ * to be included in the transaction state that is serialized in the
+ * parallel DSM. We need to temporarily escape parallel mode in order
+ * for this to be possible.
+ * For parallel SELECT (as part of non-parallel INSERT), to avoid an
+ * attempt on INSERT to acquire the FullTransactionId whilst in
+ * parallel mode, we similarly assign the FullTransactionId here.
+ */
+ if (outerPlanState(pstate)->type == T_ModifyTableState ||
+ estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ /*
+ * Assign FullTransactionId and CurrentCommandId, to be
+ * included in the transaction state that is serialized in the DSM.
+ */
+ if (outerPlanState(pstate)->type == T_ModifyTableState)
+ GetCurrentCommandId(true);
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index b399592..7e90df5 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -3903,6 +3903,36 @@ compute_parallel_worker(RelOptInfo *rel, double heap_pages, double index_pages,
}
/*
+ * Compute the number of parallel workers that should be used to insert into
+ * a relation.
+ *
+ * "max_workers" is caller's limit on the number of workers. This typically
+ * comes from a GUC.
+ */
+int
+compute_parallel_insert_worker(RelOptInfo *rel, int max_workers)
+{
+ int parallel_workers = 0;
+
+ /*
+ * If the user has set the parallel_workers reloption, use that; otherwise
+ * select a default number of workers.
+ */
+ if (rel->rel_parallel_workers != -1)
+ parallel_workers = rel->rel_parallel_workers;
+ else
+ {
+ /* TODO - smarts for computing best no. of workers for insert */
+ parallel_workers = max_workers;
+ }
+
+ /* In no case use more than caller supplied maximum number of workers */
+ parallel_workers = Min(parallel_workers, max_workers);
+
+ return parallel_workers;
+}
+
+/*
* generate_partitionwise_join_paths
* Create paths representing partitionwise join for given partitioned
* join relation.
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index cd3716d..f34b4c5 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -179,6 +179,7 @@ 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);
static double get_parallel_divisor(Path *path);
+static double get_modifytable_parallel_divisor(ModifyTablePath *path);
/*
@@ -203,6 +204,66 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ double total_rows;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ total_rows = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ total_rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+
+ /* Adjust costing for parallelism, if used. */
+ if (path->path.parallel_workers > 0)
+ {
+ double parallel_divisor = get_modifytable_parallel_divisor(path);
+
+ /* The total cost is divided among all the workers. */
+ path->path.total_cost /= parallel_divisor;
+
+ /*
+ * In the case of a parallel plan, the row count needs to represent
+ * the number of tuples processed per worker.
+ */
+ path->path.rows = clamp_row_est(total_rows / parallel_divisor);
+ }
+ else
+ {
+ path->path.rows = total_rows;
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should report zero if no RETURNING, else an average
+ * of the RETURNING tlist widths. But it's what happened historically,
+ * and improving it is a task for another day.
+ */
+ if (total_rows > 0)
+ total_size /= total_rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
@@ -5737,6 +5798,29 @@ get_parallel_divisor(Path *path)
}
/*
+ * Divisor for ModifyTable (currently only Parallel Insert).
+ * Estimate the fraction of the work that each worker will do given the
+ * number of workers budgeted for the path.
+ * TODO: Needs revising based on further experience.
+ */
+static double
+get_modifytable_parallel_divisor(ModifyTablePath *path)
+{
+ double parallel_divisor = path->path.parallel_workers;
+
+ if (parallel_leader_participation && path->returningLists != NIL)
+ {
+ double leader_contribution;
+
+ leader_contribution = 1.0 - (0.3 * path->path.parallel_workers);
+ if (leader_contribution > 0)
+ parallel_divisor += leader_contribution;
+ }
+
+ return parallel_divisor;
+}
+
+/*
* compute_bitmap_pages
*
* compute number of pages fetched from heap in bitmap heap scan.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 99278ee..cb9a6ea25 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(plan->lefttree, ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 3e2b496..9b0f293 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
/* Determine what fraction of the plan is likely to be scanned */
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
@@ -1797,7 +1798,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1845,6 +1847,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_insert_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2381,13 +2384,97 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider Parallel INSERT */
+ if (parse->commandType == CMD_INSERT &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelInsertWorkers;
+
+ parallelInsertWorkers = compute_parallel_insert_worker(current_rel, max_parallel_workers_per_gather);
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelInsertWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_insert_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2404,6 +2491,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_insert_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7573,7 +7666,43 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (root->parse->commandType == CMD_INSERT)
+ {
+ Relation relation;
+ bool hasFKs;
+ RangeTblEntry *rte;
+ List *cachedFKs;
+
+ /* Check if the target relation has foreign keys; if so, avoid
+ * creating a parallel Insert plan (because inserting into
+ * such tables would result in creation of new CommandIds, and
+ * this isn't supported by parallel workers).
+ * Similarly, avoid creating a parallel Insert plan if ON
+ * CONFLICT ... DO UPDATE ... has been specified, because
+ * parallel UPDATE is not supported.
+ * However, do allow any underlying query to be run by parallel
+ * workers in these cases.
+ */
+
+ rte = rt_fetch(root->parse->resultRelation, root->parse->rtable);
+ relation = table_open(rte->relid, NoLock);
+ cachedFKs = RelationGetFKeyList(relation);
+ hasFKs = cachedFKs != NIL;
+ table_close(relation, NoLock);
+
+ if (hasFKs || (root->parse->onConflict != NULL && root->parse->onConflict->action == ONCONFLICT_UPDATE))
+ {
+ generate_useful_gather_paths(root, rel, false);
+ /* Don't allow parallel insert */
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index dd8e2e9..062c5fc 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan->type == T_Gather || finalPlan->type == T_GatherMerge)
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (subplan->type == T_ModifyTable && ((ModifyTable *)subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = outerPlan(finalPlan)->targetlist;
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index c1fc866..4a9c3fa 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3538,11 +3538,11 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallel_workers)
{
+ ListCell *lc;
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
- ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
Assert(list_length(resultRelations) == list_length(subroots));
@@ -3557,45 +3557,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->path.pathtarget = rel->reltarget;
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
- pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
+ pathnode->path.parallel_safe = rel->consider_parallel;
+ if (rel->consider_parallel)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ foreach (lc, subpaths)
+ {
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
+ }
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallel_workers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3611,6 +3588,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index df1b43a..96295bc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -385,6 +385,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..afb8a57 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 10b6e81..1a01dcf 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -58,6 +58,8 @@ extern void generate_useful_gather_paths(PlannerInfo *root, RelOptInfo *rel,
bool override_rows);
extern int compute_parallel_worker(RelOptInfo *rel, double heap_pages,
double index_pages, int max_workers);
+extern int compute_parallel_insert_worker(RelOptInfo *rel,
+ int max_workers);
extern void create_partial_bitmap_paths(PlannerInfo *root, RelOptInfo *rel,
Path *bitmapqual);
extern void generate_partitionwise_join_paths(PlannerInfo *root,
On Mon, Sep 28, 2020 at 8:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Sep 26, 2020 at 3:30 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.Hi Bharath,
I pretty much agree with your above points.
I've attached an updated Parallel INSERT...SELECT patch, that:
- Only uses existing transaction state serialization support for
transfer of xid and cid.
- Adds a "SetCurrentCommandIdUsedForWorker" function, for setting
currentCommandIdUsed=true at the start of a parallel operation (used
for Parallel INSERT case, where we know the currentCommandId has been
assigned to the worker at the start of the parallel operation).
- Tweaks the Assert condition within "used=true" parameter case in
GetCurrentCommandId(), so that it only fires if in a parallel worker
and currentCommandId is false - refer to the updated comment in that
function.
- Does not modify any existing GetCurrentCommandId() calls.
- Does not remove any existing parallel-related asserts/checks, except
for the "cannot insert tuples in a parallel worker" error in
heap_prepare_insert(). I am still considering what to do with the
original error-check here.
[- Does not yet cater for other exclusion cases that you and Vignesh
have pointed out]This patch is mostly a lot cleaner, but does contain a possible ugly
hack, in that where it needs to call GetCurrentFullTransactionId(), it
must temporarily escape parallel-mode (recalling that parallel-mode is
set true right at the top of ExectePlan() in the cases of Parallel
INSERT/SELECT).
Thanks Greg.
In general, see a few things common to all parallel insert
cases(CTAS[1]/messages/by-id/CALj2ACWj+3H5TQqwxANZmdePEnSNxk-YAeT1c5WE184Gf75XUw@mail.gmail.com, COPY[2]/messages/by-id/CAA4eK1+kpddvvLxWm4BuG_AhVvYz8mKAEa7osxp_X0d4ZEiV=g@mail.gmail.com, INSERT INTO SELECTs):
1. Removal of "cannot insert tuples in a parallel worker" restriction
from heap_prepare_insert()
2. Each worker should be able to set currentCommandIdUsed to true.
3. The change you proposed to make in GetCurrentCommandId()'s assert condition.
Please add if I miss any other common point.
Common solutions to each of the above points would be beneficial to
all the parallel insert cases. How about having a common thread,
discussion and a common patch for all the 3 points?
@Amit Kapila @Greg Nancarrow @vignesh C Thoughts?
[1]: /messages/by-id/CALj2ACWj+3H5TQqwxANZmdePEnSNxk-YAeT1c5WE184Gf75XUw@mail.gmail.com
[2]: /messages/by-id/CAA4eK1+kpddvvLxWm4BuG_AhVvYz8mKAEa7osxp_X0d4ZEiV=g@mail.gmail.com
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Sep 28, 2020 at 4:06 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
On Mon, Sep 28, 2020 at 8:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Sep 26, 2020 at 3:30 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.Hi Bharath,
I pretty much agree with your above points.
I've attached an updated Parallel INSERT...SELECT patch, that:
- Only uses existing transaction state serialization support for
transfer of xid and cid.
- Adds a "SetCurrentCommandIdUsedForWorker" function, for setting
currentCommandIdUsed=true at the start of a parallel operation (used
for Parallel INSERT case, where we know the currentCommandId has been
assigned to the worker at the start of the parallel operation).
- Tweaks the Assert condition within "used=true" parameter case in
GetCurrentCommandId(), so that it only fires if in a parallel worker
and currentCommandId is false - refer to the updated comment in that
function.
- Does not modify any existing GetCurrentCommandId() calls.
- Does not remove any existing parallel-related asserts/checks, except
for the "cannot insert tuples in a parallel worker" error in
heap_prepare_insert(). I am still considering what to do with the
original error-check here.
[- Does not yet cater for other exclusion cases that you and Vignesh
have pointed out]This patch is mostly a lot cleaner, but does contain a possible ugly
hack, in that where it needs to call GetCurrentFullTransactionId(), it
must temporarily escape parallel-mode (recalling that parallel-mode is
set true right at the top of ExectePlan() in the cases of Parallel
INSERT/SELECT).Thanks Greg.
In general, see a few things common to all parallel insert
cases(CTAS[1], COPY[2], INSERT INTO SELECTs):
1. Removal of "cannot insert tuples in a parallel worker" restriction
from heap_prepare_insert()
2. Each worker should be able to set currentCommandIdUsed to true.
3. The change you proposed to make in GetCurrentCommandId()'s assert condition.Please add if I miss any other common point.
Common solutions to each of the above points would be beneficial to
all the parallel insert cases. How about having a common thread,
discussion and a common patch for all the 3 points?
I am not sure if that is required at this stage, lets first sort out
other parts of the design because there could be other bigger problems
which we have not thought bout yet. I have already shared some
thoughts on those points in this thread, lets first get that done and
have the basic patch ready then if required we can discuss in detail
about these points in other thread.
--
With Regards,
Amit Kapila.
On Mon, Sep 28, 2020 at 8:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Sep 26, 2020 at 3:30 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.Hi Bharath,
I pretty much agree with your above points.
I've attached an updated Parallel INSERT...SELECT patch, that:
- Only uses existing transaction state serialization support for
transfer of xid and cid.
- Adds a "SetCurrentCommandIdUsedForWorker" function, for setting
currentCommandIdUsed=true at the start of a parallel operation (used
for Parallel INSERT case, where we know the currentCommandId has been
assigned to the worker at the start of the parallel operation).
- Tweaks the Assert condition within "used=true" parameter case in
GetCurrentCommandId(), so that it only fires if in a parallel worker
and currentCommandId is false - refer to the updated comment in that
function.
- Does not modify any existing GetCurrentCommandId() calls.
- Does not remove any existing parallel-related asserts/checks, except
for the "cannot insert tuples in a parallel worker" error in
heap_prepare_insert(). I am still considering what to do with the
original error-check here.
[- Does not yet cater for other exclusion cases that you and Vignesh
have pointed out]This patch is mostly a lot cleaner, but does contain a possible ugly
hack, in that where it needs to call GetCurrentFullTransactionId(), it
must temporarily escape parallel-mode (recalling that parallel-mode is
set true right at the top of ExectePlan() in the cases of Parallel
INSERT/SELECT).
I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below
-> Gather
-> Parallel Insert
-> Parallel Seq Scan
That means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Tue, Sep 29, 2020 at 8:27 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Sep 28, 2020 at 8:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Sep 26, 2020 at 3:30 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:I further checked on full txn id and command id. Yes, these are
getting passed to workers via InitializeParallelDSM() ->
SerializeTransactionState(). I tried to summarize what we need to do
in case of parallel inserts in general i.e. parallel COPY, parallel
inserts in INSERT INTO and parallel inserts in CTAS.In the leader:
GetCurrentFullTransactionId()
GetCurrentCommandId(true)
EnterParallelMode();
InitializeParallelDSM() --> calls SerializeTransactionState()
(both full txn id and command id are serialized into parallel DSM)In the workers:
ParallelWorkerMain() --> calls StartParallelWorkerTransaction() (both
full txn id and command id are restored into workers'
CurrentTransactionState->fullTransactionId and currentCommandId)
If the parallel workers are meant for insertions, then we need to set
currentCommandIdUsed = true; Maybe we can lift the assert in
GetCurrentCommandId(), if we don't want to touch that function, then
we can have a new function GetCurrentCommandidInWorker() whose
functionality will be same as GetCurrentCommandId() without the
Assert(!IsParallelWorker());.Am I missing something?
If the above points are true, we might have to update the parallel
copy patch set, test the use cases and post separately in the parallel
copy thread in coming days.Hi Bharath,
I pretty much agree with your above points.
I've attached an updated Parallel INSERT...SELECT patch, that:
- Only uses existing transaction state serialization support for
transfer of xid and cid.
- Adds a "SetCurrentCommandIdUsedForWorker" function, for setting
currentCommandIdUsed=true at the start of a parallel operation (used
for Parallel INSERT case, where we know the currentCommandId has been
assigned to the worker at the start of the parallel operation).
- Tweaks the Assert condition within "used=true" parameter case in
GetCurrentCommandId(), so that it only fires if in a parallel worker
and currentCommandId is false - refer to the updated comment in that
function.
- Does not modify any existing GetCurrentCommandId() calls.
- Does not remove any existing parallel-related asserts/checks, except
for the "cannot insert tuples in a parallel worker" error in
heap_prepare_insert(). I am still considering what to do with the
original error-check here.
[- Does not yet cater for other exclusion cases that you and Vignesh
have pointed out]This patch is mostly a lot cleaner, but does contain a possible ugly
hack, in that where it needs to call GetCurrentFullTransactionId(), it
must temporarily escape parallel-mode (recalling that parallel-mode is
set true right at the top of ExectePlan() in the cases of Parallel
INSERT/SELECT).I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below-> Gather
-> Parallel Insert
-> Parallel Seq ScanThat means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.
I just looked into the parallel CTAS[1]/messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com patch for the same thing, and
I can see in that patch it is being handled.
[1]: /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below-> Gather
-> Parallel Insert
-> Parallel Seq ScanThat means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.I just looked into the parallel CTAS[1] patch for the same thing, and
I can see in that patch it is being handled.[1] /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
Hi Dilip,
You're right, the costing for Parallel Insert is not done and
finished, I'm still working on the costing, and haven't posted an
updated patch for it yet.
As far as cost_gather() method is concerned, for Parallel INSERT, it
can probably use the same costing approach as the CTAS patch except in
the case of a specified RETURNING clause.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Sep 30, 2020 at 7:38 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below-> Gather
-> Parallel Insert
-> Parallel Seq ScanThat means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.I just looked into the parallel CTAS[1] patch for the same thing, and
I can see in that patch it is being handled.[1] /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
Hi Dilip,
You're right, the costing for Parallel Insert is not done and
finished, I'm still working on the costing, and haven't posted an
updated patch for it yet.
Okay.
As far as cost_gather() method is concerned, for Parallel INSERT, it
can probably use the same costing approach as the CTAS patch except in
the case of a specified RETURNING clause.
Yeah right. I did not think about the returning part.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Sep 30, 2020 at 7:38 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below-> Gather
-> Parallel Insert
-> Parallel Seq ScanThat means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.I just looked into the parallel CTAS[1] patch for the same thing, and
I can see in that patch it is being handled.[1] /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
Hi Dilip,
You're right, the costing for Parallel Insert is not done and
finished, I'm still working on the costing, and haven't posted an
updated patch for it yet.
As far as cost_gather() method is concerned, for Parallel INSERT, it
can probably use the same costing approach as the CTAS patch except in
the case of a specified RETURNING clause.
I have one question which is common to both this patch and parallel
inserts in CTAS[1]/messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com, do we need to skip creating tuple
queues(ExecParallelSetupTupleQueues) as we don't have any tuples
that's being shared from workers to leader? Put it another way, do we
use the tuple queue for sharing any info other than tuples from
workers to leader?
[1]: /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Oct 5, 2020 at 4:26 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
On Wed, Sep 30, 2020 at 7:38 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
I think you still need to work on the costing part, basically if we
are parallelizing whole insert then plan is like below-> Gather
-> Parallel Insert
-> Parallel Seq ScanThat means the tuple we are selecting via scan are not sent back to
the gather node, so in cost_gather we need to see if it is for the
INSERT then there is no row transferred through the parallel queue
that mean we need not to pay any parallel tuple cost.I just looked into the parallel CTAS[1] patch for the same thing, and
I can see in that patch it is being handled.[1] /messages/by-id/CALj2ACWFq6Z4_jd9RPByURB8-Y8wccQWzLf+0-Jg+KYT7ZO-Ug@mail.gmail.com
Hi Dilip,
You're right, the costing for Parallel Insert is not done and
finished, I'm still working on the costing, and haven't posted an
updated patch for it yet.
As far as cost_gather() method is concerned, for Parallel INSERT, it
can probably use the same costing approach as the CTAS patch except in
the case of a specified RETURNING clause.I have one question which is common to both this patch and parallel
inserts in CTAS[1], do we need to skip creating tuple
queues(ExecParallelSetupTupleQueues) as we don't have any tuples
that's being shared from workers to leader? Put it another way, do we
use the tuple queue for sharing any info other than tuples from
workers to leader?
Ideally, we don't need the tuple queue unless we want to transfer the
tuple to the gather node.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Mon, Oct 5, 2020 at 4:26 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
On Wed, Sep 30, 2020 at 7:38 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
I have one question which is common to both this patch and parallel
inserts in CTAS[1], do we need to skip creating tuple
queues(ExecParallelSetupTupleQueues) as we don't have any tuples
that's being shared from workers to leader?
As far as this patch is concerned we might need to return tuples when
there is a Returning clause. I think for the cases where we don't need
to return tuples we might want to skip creating these queues if it is
feasible without too many changes.
--
With Regards,
Amit Kapila.
On Mon, Oct 5, 2020 at 4:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Oct 5, 2020 at 4:26 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:On Wed, Sep 30, 2020 at 7:38 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
I have one question which is common to both this patch and parallel
inserts in CTAS[1], do we need to skip creating tuple
queues(ExecParallelSetupTupleQueues) as we don't have any tuples
that's being shared from workers to leader?As far as this patch is concerned we might need to return tuples when
there is a Returning clause. I think for the cases where we don't need
to return tuples we might want to skip creating these queues if it is
feasible without too many changes.
+1
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Mon, Oct 5, 2020 at 10:36 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
I have one question which is common to both this patch and parallel
inserts in CTAS[1], do we need to skip creating tuple
queues(ExecParallelSetupTupleQueues) as we don't have any tuples
that's being shared from workers to leader?As far as this patch is concerned we might need to return tuples when
there is a Returning clause. I think for the cases where we don't need
to return tuples we might want to skip creating these queues if it is
feasible without too many changes.
Hi Dilip,
You're right. I've included that in my latest version of the patch (so
Gather should only start tuple queues in the case of parallel SELECT
or parallel INSERT with a RETURNING clause).
Other functionality updated includes:
- Added more necessary exclusions for Parallel INSERT INTO ... SELECT
... (but allowing underlying query to still be parallel):
- non-parallel-safe triggers
- non-parallel-safe default and check expressions
- foreign tables
- temporary tables
- Added support for before/after statement-level INSERT triggers
(can't allow parallel workers to execute these)
- Adjusted cost of Gather node, for when RETURNING clause is not specified
I have not found issues with partition tables (yet) or toast column values.
Also, I have attached a separate patch (requested by Andres Freund)
that just allows the underlying SELECT part of "INSERT INTO ... SELECT
..." to be parallel.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
0004-ParallelInsertSelect.patchapplication/octet-stream; name=0004-ParallelInsertSelect.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..94c8507 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
-
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
tup->t_data->t_infomask |= HEAP_XMAX_INVALID;
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..8c69931 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -764,12 +778,13 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed
+ * if currentCommandIdUsed was already true at the start of the
+ * parallel operation (by way of SetCurrentCommandIdUsed()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision
+ * for communicating this back to the leader.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2e27e26..0f3bd82 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -173,7 +173,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* against performing unsafe operations in parallel mode, but this gives a
* more user-friendly error message.
*/
- if ((XactReadOnly || IsInParallelMode()) &&
+ if ((XactReadOnly || (IsInParallelMode() && queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 382e78f..0e62554 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,18 +175,20 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ estate->es_plannedstmt->commandType == CMD_INSERT);
+ pstmt->commandType = IsA(plan, ModifyTable) ? CMD_INSERT : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
- pstmt->hasModifyingCTE = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
+ pstmt->hasModifyingCTE = estate->es_plannedstmt->hasModifyingCTE;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
pstmt->dependsOnRole = false;
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
- pstmt->rootResultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
+ pstmt->rootResultRelations = estate->es_plannedstmt->rootResultRelations;
pstmt->appendRelations = NIL;
/*
@@ -591,6 +595,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -676,6 +681,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -765,6 +778,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Allocate space for each worker's returned "# of tuples processed" count. */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1153,6 +1179,16 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ * This is currently done only in the case of parallel INSERT.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1380,6 +1416,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1401,6 +1438,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || queryDesc->operation == CMD_INSERT);
+ if (queryDesc->operation == CMD_INSERT)
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of
+ * the parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1459,6 +1506,13 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (queryDesc->operation == CMD_INSERT)
+ {
+ /* Report the # of tuples processed during parallel INSERT execution. */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..7c2aa52 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelInsertLeader = false;
+ bool isParallelInsertWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelInsertLeader = nodeModifyTableState->operation == CMD_INSERT;
+ isParallelInsertWithReturning = isParallelInsertLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -166,6 +180,28 @@ ExecGather(PlanState *pstate)
{
ParallelContext *pcxt;
+ /* For parallel INSERT, assign FullTransactionId and CurrentCommandId,
+ * to be included in the transaction state that is serialized in the
+ * parallel DSM. We need to temporarily escape parallel mode in order
+ * for this to be possible.
+ * For parallel SELECT (as part of non-parallel INSERT), to avoid an
+ * attempt on INSERT to assign the FullTransactionId whilst in
+ * parallel mode, we similarly assign the FullTransactionId here.
+ */
+ if (isParallelInsertLeader || estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ /*
+ * Assign FullTransactionId and CurrentCommandId, to be
+ * included in the transaction state that is serialized in the DSM.
+ */
+ if (isParallelInsertLeader)
+ GetCurrentCommandId(true);
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
@@ -178,6 +214,25 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelInsertLeader)
+ {
+ /* For Parallel INSERT, if there are BEFORE STATEMENT triggers,
+ * these must be fired by the leader, not the parallel workers.
+ */
+ if (nodeModifyTableState->fireBSTriggers)
+ {
+ fireBSTriggers(nodeModifyTableState);
+ nodeModifyTableState->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local
+ * plan execution (ModifyTable processing). These will be
+ * fired at end of Gather processing.
+ */
+ nodeModifyTableState->fireASTriggers = false;
+ }
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +243,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelInsertLeader && !isParallelInsertWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +255,10 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a parallel INSERT
+ * without a RETURNING clause - no readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +266,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -418,14 +476,25 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ bool isParallelInsertLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ castNode(ModifyTableState, outerPlanState(node))->operation == CMD_INSERT;
+ if (isParallelInsertLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /* For Parallel INSERT, if there are AFTER STATEMENT triggers, these must be
+ * fired by the leader, not the parallel workers.
+ */
+ ModifyTableState *nodeMTS = castNode(ModifyTableState, outerPlanState(node));
+ fireASTriggers(nodeMTS);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeGatherMerge.c b/src/backend/executor/nodeGatherMerge.c
index 4712934..cc197dd 100644
--- a/src/backend/executor/nodeGatherMerge.c
+++ b/src/backend/executor/nodeGatherMerge.c
@@ -1,4 +1,4 @@
-/*-------------------------------------------------------------------------
+/*------------------------------------------------------------------------
*
* nodeGatherMerge.c
* Scan a plan in multiple workers, and do order-preserving merge.
@@ -210,6 +210,21 @@ ExecGatherMerge(PlanState *pstate)
{
ParallelContext *pcxt;
+ if (estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ /*
+ * We need to avoid an attempt on INSERT to assign a
+ * FullTransactionId whilst in parallel mode (which is in
+ * effect due to the underlying parallel query) - so the
+ * FullTransactionId is assigned here. Parallel mode must
+ * be temporarily escaped in order for this to be possible.
+ */
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 9812089..1e97974 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1734,7 +1735,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate,
/*
* Process BEFORE EACH STATEMENT triggers
*/
-static void
+void
fireBSTriggers(ModifyTableState *node)
{
ModifyTable *plan = (ModifyTable *) node->ps.plan;
@@ -1793,7 +1794,7 @@ getTargetResultRelInfo(ModifyTableState *node)
/*
* Process AFTER EACH STATEMENT triggers
*/
-static void
+void
fireASTriggers(ModifyTableState *node)
{
ModifyTable *plan = (ModifyTable *) node->ps.plan;
@@ -2281,7 +2282,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2335,7 +2340,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* call ExecInitNode on each of the plans to be executed and save the
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index cd3716d..7786624 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -179,6 +179,7 @@ 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);
static double get_parallel_divisor(Path *path);
+static double get_modifytable_parallel_divisor(ModifyTablePath *path);
/*
@@ -203,6 +204,66 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ double total_rows;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ total_rows = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ total_rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+
+ /* Adjust costing for parallelism, if used. */
+ if (path->path.parallel_workers > 0)
+ {
+ double parallel_divisor = get_modifytable_parallel_divisor(path);
+
+ /* The total cost is divided among all the workers. */
+ path->path.total_cost /= parallel_divisor;
+
+ /*
+ * In the case of a parallel plan, the row count needs to represent
+ * the number of tuples processed per worker.
+ */
+ path->path.rows = clamp_row_est(total_rows / parallel_divisor);
+ }
+ else
+ {
+ path->path.rows = total_rows;
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should report zero if no RETURNING, else an average
+ * of the RETURNING tlist widths. But it's what happened historically,
+ * and improving it is a task for another day.
+ */
+ if (total_rows > 0)
+ total_size /= total_rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
@@ -383,7 +444,21 @@ cost_gather(GatherPath *path, PlannerInfo *root,
/* Parallel setup and communication cost. */
startup_cost += parallel_setup_cost;
- run_cost += parallel_tuple_cost * path->path.rows;
+
+ /*
+ * For Parallel INSERT, provided no tuples are returned from workers
+ * to gather/leader node, don't add a cost-per-row, as each worker
+ * parallelly inserts the tuples that result from its chunk of plan
+ * execution. This change may make the parallel plan cheap among all
+ * other plans, and influence the planner to consider this parallel
+ * plan.
+ */
+ if (!(IsA(path->subpath, ModifyTablePath) &&
+ castNode(ModifyTablePath, path->subpath)->operation == CMD_INSERT &&
+ castNode(ModifyTablePath, path->subpath)->returningLists != NULL))
+ {
+ run_cost += parallel_tuple_cost * path->path.rows;
+ }
path->path.startup_cost = startup_cost;
path->path.total_cost = (startup_cost + run_cost);
@@ -5737,6 +5812,29 @@ get_parallel_divisor(Path *path)
}
/*
+ * Divisor for ModifyTable (currently only Parallel Insert).
+ * Estimate the fraction of the work that each worker will do given the
+ * number of workers budgeted for the path.
+ * TODO: Needs revising based on further experience.
+ */
+static double
+get_modifytable_parallel_divisor(ModifyTablePath *path)
+{
+ double parallel_divisor = path->path.parallel_workers;
+
+ if (parallel_leader_participation && path->returningLists != NIL)
+ {
+ double leader_contribution;
+
+ leader_contribution = 1.0 - (0.3 * path->path.parallel_workers);
+ if (leader_contribution > 0)
+ parallel_divisor += leader_contribution;
+ }
+
+ return parallel_divisor;
+}
+
+/*
* compute_bitmap_pages
*
* compute number of pages fetched from heap in bitmap heap scan.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 3d7a4e3..825896b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f331f82..9573a28 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -28,6 +28,7 @@
#include "catalog/pg_inherits.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
#include "foreign/fdwapi.h"
@@ -58,6 +59,7 @@
#include "parser/parse_agg.h"
#include "parser/parsetree.h"
#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
#include "storage/dsm_impl.h"
#include "utils/lsyscache.h"
@@ -337,7 +339,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -371,6 +373,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
/* Determine what fraction of the plan is likely to be scanned */
@@ -425,7 +428,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
@@ -1797,7 +1800,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1845,6 +1849,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_insert_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2381,13 +2386,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider Parallel INSERT */
+ if (parse->commandType == CMD_INSERT &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelInsertWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel INSERT, it
+ * seems resonable to use the same number of workers as estimated
+ * for the underlying query.
+ */
+ parallelInsertWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelInsertWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_insert_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2404,6 +2498,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_insert_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7355,6 +7455,163 @@ can_partial_agg(PlannerInfo *root, const AggClauseCosts *agg_costs)
}
/*
+ * IsTriggerDataParallelInsertSafe
+ *
+ * Checks if the specified trigger data is parallel safe.
+ * Returns false if any one of the triggers are not safe
+ * for parallel insert.
+ */
+static pg_attribute_always_inline bool
+IsTriggerDataParallelInsertSafe(TriggerDesc *trigdesc)
+{
+ int i;
+
+ /*
+ * Can't support execution of the following triggers during
+ * insert by parallel workers:
+ * - before/after statement trigger
+ * - before/after row trigger
+ * - instead of trigger
+ * - transition table trigger
+ * Note however that for parallel INSERT, any before/after
+ * Insert statement triggers are executed in the leader only
+ * (not the workers), so checks for those types of triggers
+ * are not included here.
+ */
+ if (trigdesc != NULL &&
+ (trigdesc->trig_insert_instead_row ||
+ trigdesc->trig_insert_before_row ||
+ trigdesc->trig_insert_after_row ||
+ trigdesc->trig_insert_new_table))
+ {
+ return false;
+ }
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (func_parallel(trigger->tgfoid) != PROPARALLEL_SAFE)
+ return false;
+
+ /* If the trigger type is RI_TRIGGER_FK, this indicates a FK
+ * exists in the relation, and this is not parallel-safe for
+ * insert, as it would result in creation of new CommandIds,
+ * and this isn't supported by parallel workers.
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ return false;
+ }
+
+ return true;
+}
+
+/*
+ * IsParallelInsertSafe
+ *
+ * Determines whether a specified INSERT statement is parallel safe.
+ */
+static bool
+IsParallelInsertSafe(Query *parse)
+{
+ Relation rel;
+ RangeTblEntry *rte;
+ TupleDesc tupdesc;
+ int attnum;
+
+ /*
+ * It's not safe to create a parallel Insert plan if
+ * ON CONFLICT ... DO UPDATE ... has been specified, because
+ * parallel UPDATE is not supported.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ /*
+ * We can't support insert by parallel workers on certain table types:
+ * - foreign table (no FDW API for supporting parallel insert)
+ * - temporary table (may not be accessible by parallel workers)
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, NoLock);
+ return false;
+ }
+
+ /* If any triggers, check they are parallel safe. */
+ if (rel->trigdesc != NULL &&
+ !IsTriggerDataParallelInsertSafe(rel->trigdesc))
+ {
+ table_close(rel, NoLock);
+ return false;
+ }
+
+ /*
+ * Check if any of the columns has a non-parallel-safe
+ * volatile default expression.
+ */
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Expr *defexpr;
+ bool isVolatileExpr;
+
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ if (att->atthasdef)
+ {
+ defexpr = (Expr *)build_column_default(rel, attnum + 1);
+
+ /* Run the expression through planner */
+ defexpr = expression_planner(defexpr);
+
+ isVolatileExpr = contain_volatile_functions((Node *)defexpr);
+ if (isVolatileExpr &&
+ (max_parallel_hazard((Query *)defexpr)) != PROPARALLEL_SAFE)
+ {
+ table_close(rel, NoLock);
+ return false;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *checkExpr = stringToNode(check->ccbin);
+ bool isVolatileExpr = contain_volatile_functions((Node *)checkExpr);
+ if (isVolatileExpr &&
+ (max_parallel_hazard((Query *)checkExpr)) != PROPARALLEL_SAFE)
+ {
+ table_close(rel, NoLock);
+ return false;
+ }
+ }
+ }
+
+ table_close(rel, NoLock);
+ return true;
+}
+
+/*
* apply_scanjoin_target_to_paths
*
* Adjust the final scan/join relation, and recursively all of its children,
@@ -7573,7 +7830,24 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (root->parse->commandType == CMD_INSERT)
+ {
+ if (!IsParallelInsertSafe(root->parse))
+ {
+ /*
+ * Don't allow parallel insert bacause it's not safe, but do
+ * allow any underlying query to be run by parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index dd8e2e9..e9dcd30 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = outerPlan(finalPlan)->targetlist;
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index c1fc866..4a9c3fa 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3538,11 +3538,11 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallel_workers)
{
+ ListCell *lc;
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
- ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
Assert(list_length(resultRelations) == list_length(subroots));
@@ -3557,45 +3557,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->path.pathtarget = rel->reltarget;
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
- pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
+ pathnode->path.parallel_safe = rel->consider_parallel;
+ if (rel->consider_parallel)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ foreach (lc, subpaths)
+ {
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
+ }
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallel_workers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3611,6 +3588,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index df1b43a..96295bc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -385,6 +385,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..afb8a57 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 4ec4ebd..fbddee2 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -20,5 +20,6 @@ extern void ExecComputeStoredGenerated(EState *estate, TupleTableSlot *slot, Cmd
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggers(ModifyTableState *node);
+extern void fireASTriggers(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ef448d6..cc744b6 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1163,7 +1163,8 @@ typedef struct ModifyTableState
* table root) */
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
0001-InsertParallelSelect.patchapplication/octet-stream; name=0001-InsertParallelSelect.patchDownload
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..ff08f37 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -166,6 +166,23 @@ ExecGather(PlanState *pstate)
{
ParallelContext *pcxt;
+ /*
+ * We need to avoid an attempt on INSERT to assign a
+ * FullTransactionId whilst in parallel mode (which is in
+ * effect due to the underlying parallel query) - so the
+ * FullTransactionId is assigned here. Parallel mode must
+ * be temporarily escaped in order for this to be possible.
+ * The FullTransactionId will be included in the transaction
+ * state that is serialized in the parallel DSM.
+ */
+ if (estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
diff --git a/src/backend/executor/nodeGatherMerge.c b/src/backend/executor/nodeGatherMerge.c
index 4712934..cc197dd 100644
--- a/src/backend/executor/nodeGatherMerge.c
+++ b/src/backend/executor/nodeGatherMerge.c
@@ -1,4 +1,4 @@
-/*-------------------------------------------------------------------------
+/*------------------------------------------------------------------------
*
* nodeGatherMerge.c
* Scan a plan in multiple workers, and do order-preserving merge.
@@ -210,6 +210,21 @@ ExecGatherMerge(PlanState *pstate)
{
ParallelContext *pcxt;
+ if (estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ /*
+ * We need to avoid an attempt on INSERT to assign a
+ * FullTransactionId whilst in parallel mode (which is in
+ * effect due to the underlying parallel query) - so the
+ * FullTransactionId is assigned here. Parallel mode must
+ * be temporarily escaped in order for this to be possible.
+ */
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f331f82..fd5b47a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
/* Determine what fraction of the plan is likely to be scanned */
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
On Tue, Oct 6, 2020 at 3:08 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I have not found issues with partition tables (yet) or toast column values.
I think for toast column values there may not be a problem as each
parallel worker inserts toast column values individually.
But the problem may arise if a partitioned table has foreign table as
a partition, I think we can not allow parallelism for this case too,
but it's hard to determine ahead of time whether a table has a foreign
partition.(See [1]else { /* * For partitioned tables, we may still be able to perform bulk * inserts. However, the possibility of this depends on which types * of triggers exist on the partition. We must disable bulk inserts * if the partition is a foreign table or it has any before row insert * or insert instead triggers (same as we checked above for the parent * table). Since the partition's resultRelInfos are initialized only * when we actually need to insert the first tuple into them, we must * have the intermediate insert method of CIM_MULTI_CONDITIONAL to * flag that we must later determine if we can use bulk-inserts for * the partition being inserted into. */ if (proute) insertMethod = CIM_MULTI_CONDITIONAL; in copy.c)
- Added support for before/after statement-level INSERT triggers
(can't allow parallel workers to execute these)
I think we can allow parallelism for before statement level-triggers.
Leader can execute this trigger and go for parallel inserts.
How about before row, after row, instead row, new table type triggers?
[1]: else { /* * For partitioned tables, we may still be able to perform bulk * inserts. However, the possibility of this depends on which types * of triggers exist on the partition. We must disable bulk inserts * if the partition is a foreign table or it has any before row insert * or insert instead triggers (same as we checked above for the parent * table). Since the partition's resultRelInfos are initialized only * when we actually need to insert the first tuple into them, we must * have the intermediate insert method of CIM_MULTI_CONDITIONAL to * flag that we must later determine if we can use bulk-inserts for * the partition being inserted into. */ if (proute) insertMethod = CIM_MULTI_CONDITIONAL;
else
{
/*
* For partitioned tables, we may still be able to perform bulk
* inserts. However, the possibility of this depends on which types
* of triggers exist on the partition. We must disable bulk inserts
* if the partition is a foreign table or it has any before row insert
* or insert instead triggers (same as we checked above for the parent
* table). Since the partition's resultRelInfos are initialized only
* when we actually need to insert the first tuple into them, we must
* have the intermediate insert method of CIM_MULTI_CONDITIONAL to
* flag that we must later determine if we can use bulk-inserts for
* the partition being inserted into.
*/
if (proute)
insertMethod = CIM_MULTI_CONDITIONAL;
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Oct 6, 2020 at 9:10 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
But the problem may arise if a partitioned table has foreign table as
a partition, I think we can not allow parallelism for this case too,
but it's hard to determine ahead of time whether a table has a foreign
partition.(See [1] in copy.c)
Thanks, I had seen that as a potential issue when scanning the code,
but had forgotten to note it. I'll check your code again.
- Added support for before/after statement-level INSERT triggers
(can't allow parallel workers to execute these)I think we can allow parallelism for before statement level-triggers.
Leader can execute this trigger and go for parallel inserts.
My attached patch implements the before/after statement-level trigger
invocation.
(For INSERT INTO ... SELECT... case, it needs to account for parallel
and non-parallel INSERT, and also the fact that, as the patch
currently stands, the leader also participates in a parallel INSERT -
so I found it necessary to invoke those triggers at the Gather node
level in that case).
How about before row, after row, instead row, new table type triggers?
My attached patch does not allow parallel INSERT if there are any
row-level triggers (as the trigger functions could see a different and
unpredictable table state compared to non-parallel INSERT, even if
otherwise parallel-safe).
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Oct 6, 2020 at 4:13 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Oct 6, 2020 at 9:10 PM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:But the problem may arise if a partitioned table has foreign table as
a partition, I think we can not allow parallelism for this case too,
but it's hard to determine ahead of time whether a table has a foreign
partition.(See [1] in copy.c)Thanks, I had seen that as a potential issue when scanning the code,
but had forgotten to note it. I'll check your code again.
In parallel, we are not doing anything(due to the same reason
explained in above comment) to find whether there is a foreign
partition or not while deciding to go with parallel/non-parallel copy,
we are just throwing an error during the first tuple insertion into
the partition.
errmsg("cannot perform PARALLEL COPY if partition has BEFORE/INSTEAD
OF triggers, or if the partition is foreign partition"),
errhint("Try COPY without PARALLEL option")));
- Added support for before/after statement-level INSERT triggers
(can't allow parallel workers to execute these)I think we can allow parallelism for before statement level-triggers.
Leader can execute this trigger and go for parallel inserts.My attached patch implements the before/after statement-level trigger
invocation.
(For INSERT INTO ... SELECT... case, it needs to account for parallel
and non-parallel INSERT, and also the fact that, as the patch
currently stands, the leader also participates in a parallel INSERT -
so I found it necessary to invoke those triggers at the Gather node
level in that case).
Allowing the leader to execute before statement triggers at Gather
node level before invoking the parallel plan and then parallel inserts
makes sense. But if there are any after statement triggers, there may
come transition tables, see Amit's findings under Case-1 in [1]/messages/by-id/CAA4eK1+ANNEaMJCCXm4naweP5PLY6LhJMvGo_V7-Pnfbh6GsOA@mail.gmail.com and we
must disable parallelism in that case.
[1]: /messages/by-id/CAA4eK1+ANNEaMJCCXm4naweP5PLY6LhJMvGo_V7-Pnfbh6GsOA@mail.gmail.com
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Oct 7, 2020 at 12:40 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
In parallel, we are not doing anything(due to the same reason
explained in above comment) to find whether there is a foreign
partition or not while deciding to go with parallel/non-parallel copy,
we are just throwing an error during the first tuple insertion into
the partition.errmsg("cannot perform PARALLEL COPY if partition has BEFORE/INSTEAD
OF triggers, or if the partition is foreign partition"),
errhint("Try COPY without PARALLEL option")));
I may well need to do something similar for parallel INSERT, but I'm
kind of surprised it can't be detected earlier (?).
Will need to further test this.
Allowing the leader to execute before statement triggers at Gather
node level before invoking the parallel plan and then parallel inserts
makes sense. But if there are any after statement triggers, there may
come transition tables, see Amit's findings under Case-1 in [1] and we
must disable parallelism in that case.[1] - /messages/by-id/CAA4eK1+ANNEaMJCCXm4naweP5PLY6LhJMvGo_V7-Pnfbh6GsOA@mail.gmail.com
The patch I last posted for parallel INSERT does detect use of
transition tables in this case (trigdesc->trig_insert_new_table) and
disables INSERT parallelism (I tested it against Amit's example), yet
still otherwise allows AFTER STATEMENT triggers for parallel INSERT.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Oct 7, 2020 at 12:40 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:
In parallel, we are not doing anything(due to the same reason
explained in above comment) to find whether there is a foreign
partition or not while deciding to go with parallel/non-parallel copy,
we are just throwing an error during the first tuple insertion into
the partition.errmsg("cannot perform PARALLEL COPY if partition has BEFORE/INSTEAD
OF triggers, or if the partition is foreign partition"),
errhint("Try COPY without PARALLEL option")));
I'm wondering whether code similar to the following can safely be used
to detect a foreign partition:
if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
int i;
PartitionDesc pd = RelationGetPartitionDesc(rel);
for (i = 0; i < pd->nparts; i++)
{
if (get_rel_relkind(pd->oids[i]) == RELKIND_FOREIGN_TABLE)
{
table_close(rel, NoLock);
return false;
}
}
}
Thoughts?
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Oct 7, 2020 at 7:25 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Oct 7, 2020 at 12:40 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:In parallel, we are not doing anything(due to the same reason
explained in above comment) to find whether there is a foreign
partition or not while deciding to go with parallel/non-parallel copy,
we are just throwing an error during the first tuple insertion into
the partition.errmsg("cannot perform PARALLEL COPY if partition has BEFORE/INSTEAD
OF triggers, or if the partition is foreign partition"),
errhint("Try COPY without PARALLEL option")));I'm wondering whether code similar to the following can safely be used
to detect a foreign partition:if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
int i;
PartitionDesc pd = RelationGetPartitionDesc(rel);
for (i = 0; i < pd->nparts; i++)
{
if (get_rel_relkind(pd->oids[i]) == RELKIND_FOREIGN_TABLE)
{
table_close(rel, NoLock);
return false;
}
}
}
Actually, the addition of this kind of check is still not good enough.
Partitions can have their own constraints, triggers, column default
expressions etc. and a partition itself can be partitioned.
I've written code to recursively walk the partitions and do all the
various checks for parallel-insert-safety as before, but it's doing a
fair bit of work.
Any other idea of dealing with this? Seems it can't be avoided if you
want to support partitioned tables and partitions.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Oct 8, 2020 at 1:42 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Oct 7, 2020 at 7:25 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Oct 7, 2020 at 12:40 AM Bharath Rupireddy
<bharath.rupireddyforpostgres@gmail.com> wrote:In parallel, we are not doing anything(due to the same reason
explained in above comment) to find whether there is a foreign
partition or not while deciding to go with parallel/non-parallel copy,
we are just throwing an error during the first tuple insertion into
the partition.errmsg("cannot perform PARALLEL COPY if partition has BEFORE/INSTEAD
OF triggers, or if the partition is foreign partition"),
errhint("Try COPY without PARALLEL option")));I'm wondering whether code similar to the following can safely be used
to detect a foreign partition:if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
int i;
PartitionDesc pd = RelationGetPartitionDesc(rel);
for (i = 0; i < pd->nparts; i++)
{
if (get_rel_relkind(pd->oids[i]) == RELKIND_FOREIGN_TABLE)
{
table_close(rel, NoLock);
return false;
}
}
}Actually, the addition of this kind of check is still not good enough.
Partitions can have their own constraints, triggers, column default
expressions etc. and a partition itself can be partitioned.
I've written code to recursively walk the partitions and do all the
various checks for parallel-insert-safety as before, but it's doing a
fair bit of work.
Any other idea of dealing with this? Seems it can't be avoided if you
want to support partitioned tables and partitions.
IMHO, it's good not to do all of this recursive checking right now,
which may complicate the code or may restrict the performance gain.
Having said that, in future we may have to do something about it.
Others may have better opinions on this point.
With Regards,
Bharath Rupireddy.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Oct 6, 2020 at 10:38 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
+ if (estate->es_plannedstmt->commandType == CMD_INSERT)
...
+ if ((XactReadOnly || (IsInParallelMode() &&
queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
...
+ isParallelInsertLeader = nodeModifyTableState->operation == CMD_INSERT;
...
One thing I noticed is that you have logic, variable names and
assertions all over the tree that assume that we can only do parallel
*inserts*. I agree 100% with your plan to make Parallel Insert work
first, it is an excellent goal and if we get it in it'll be a headline
feature of PG14 (along with COPY etc). That said, I wonder if it
would make sense to use more general naming (isParallelModifyLeader?),
be more liberal where you really mean "is it DML", and find a way to
centralise the logic about which DML commands types are currently
allowed (ie insert only for now) for assertions and error checks etc,
so that in future we don't have to go around and change all these
places and rename things again and again.
While contemplating that, I couldn't resist taking a swing at the main
(?) show stopper for Parallel Update and Parallel Delete, judging by
various clues left in code comments by Robert: combo command IDs
created by other processes. Here's a rapid prototype to make that
work (though perhaps not as efficiently as we'd want, not sure). With
that in place, I wonder what else we'd need to extend your patch to
cover all three operations... it can't be much! Of course I don't
want to derail your work on Parallel Insert, I'm just providing some
motivation for my comments on the (IMHO) shortsightedness of some of
the coding.
PS Why not use git format-patch to create patches?
Attachments:
0001-Coordinate-combo-command-IDs-with-parallel-workers.patchtext/x-patch; charset=US-ASCII; name=0001-Coordinate-combo-command-IDs-with-parallel-workers.patchDownload
From ad2b5e07a09603b09859dfcbde6addd51096cbb0 Mon Sep 17 00:00:00 2001
From: Thomas Munro <thomas.munro@gmail.com>
Date: Fri, 9 Oct 2020 00:27:07 +1300
Subject: [PATCH] Coordinate combo command IDs with parallel workers.
Previously, we would serialize the leader's combo command IDs and
restore a read-only copy of them in worker processes, not allowing
updates. Instead, migrate them into shared memory, in preparation for
parallel update/delete queries where new combo command IDs might need to
be created in any process and visible to others.
XXX This design causes every backend to maintain its own deduplication
hash table, and requires a shared lock to look up any combocid. Both
policies could be reconsidered, basically a memory size vs locking
tradeoff. Need some experience/profiling of real work to see how much
any of this really matters.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV%3DqpFJrR3AcrTS3g%40mail.gmail.com
---
src/backend/access/common/session.c | 28 +-
src/backend/access/heap/heapam.c | 10 -
src/backend/access/transam/README.parallel | 6 -
src/backend/access/transam/parallel.c | 14 -
src/backend/storage/lmgr/lwlock.c | 2 +
src/backend/utils/time/combocid.c | 290 ++++++++++++++++-----
src/include/access/session.h | 20 +-
src/include/storage/lwlock.h | 1 +
src/include/utils/combocid.h | 8 +-
9 files changed, 275 insertions(+), 104 deletions(-)
diff --git a/src/backend/access/common/session.c b/src/backend/access/common/session.c
index 0ec61d48a2..7e1bffb680 100644
--- a/src/backend/access/common/session.c
+++ b/src/backend/access/common/session.c
@@ -23,6 +23,7 @@
#include "access/session.h"
#include "storage/lwlock.h"
#include "storage/shm_toc.h"
+#include "utils/combocid.h"
#include "utils/memutils.h"
#include "utils/typcache.h"
@@ -43,6 +44,7 @@
*/
#define SESSION_KEY_DSA UINT64CONST(0xFFFFFFFFFFFF0001)
#define SESSION_KEY_RECORD_TYPMOD_REGISTRY UINT64CONST(0xFFFFFFFFFFFF0002)
+#define SESSION_KEY_FIXED UINT64CONST(0xFFFFFFFFFFFF0003)
/* This backend's current session. */
Session *CurrentSession = NULL;
@@ -74,8 +76,10 @@ GetSessionDsmHandle(void)
dsm_segment *seg;
size_t typmod_registry_size;
size_t size;
+ void *fixed_space;
void *dsa_space;
void *typmod_registry_space;
+ SessionFixed *fixed;
dsa_area *dsa;
MemoryContext old_context;
@@ -91,6 +95,10 @@ GetSessionDsmHandle(void)
old_context = MemoryContextSwitchTo(TopMemoryContext);
shm_toc_initialize_estimator(&estimator);
+ /* Estimate size for the fixed-sized per-session state. */
+ shm_toc_estimate_keys(&estimator, 1);
+ shm_toc_estimate_chunk(&estimator, sizeof(SessionFixed));
+
/* Estimate space for the per-session DSA area. */
shm_toc_estimate_keys(&estimator, 1);
shm_toc_estimate_chunk(&estimator, SESSION_DSA_SIZE);
@@ -113,6 +121,14 @@ GetSessionDsmHandle(void)
dsm_segment_address(seg),
size);
+ /* Create the simple fixed-sized session state. */
+ fixed_space = shm_toc_allocate(toc, sizeof(SessionFixed));
+ fixed = (SessionFixed *) fixed_space;
+ memset(fixed, 0, sizeof(*fixed));
+ LWLockInitialize(&fixed->shared_combocid_lock, LWTRANCHE_SHARED_COMBOCID);
+ shm_toc_insert(toc, SESSION_KEY_FIXED, fixed_space);
+ CurrentSession->fixed = fixed;
+
/* Create per-session DSA area. */
dsa_space = shm_toc_allocate(toc, SESSION_DSA_SIZE);
dsa = dsa_create_in_place(dsa_space,
@@ -121,7 +137,6 @@ GetSessionDsmHandle(void)
seg);
shm_toc_insert(toc, SESSION_KEY_DSA, dsa_space);
-
/* Create session-scoped shared record typmod registry. */
typmod_registry_space = shm_toc_allocate(toc, typmod_registry_size);
SharedRecordTypmodRegistryInit((SharedRecordTypmodRegistry *)
@@ -129,6 +144,9 @@ GetSessionDsmHandle(void)
shm_toc_insert(toc, SESSION_KEY_RECORD_TYPMOD_REGISTRY,
typmod_registry_space);
+ /* Initialize shared commmand ids. */
+ SharedComboCidRegistryInit(seg, dsa);
+
/*
* If we got this far, we can pin the shared memory so it stays mapped for
* the rest of this backend's life. If we don't make it this far, cleanup
@@ -156,6 +174,7 @@ AttachSession(dsm_handle handle)
{
dsm_segment *seg;
shm_toc *toc;
+ void *fixed_space;
void *dsa_space;
void *typmod_registry_space;
dsa_area *dsa;
@@ -177,12 +196,19 @@ AttachSession(dsm_handle handle)
CurrentSession->segment = seg;
CurrentSession->area = dsa;
+ /* Attach to the "fixed sized" data region. */
+ fixed_space = shm_toc_lookup(toc, SESSION_KEY_FIXED, false);
+ CurrentSession->fixed = (SessionFixed *) fixed_space;
+
/* Attach to the shared record typmod registry. */
typmod_registry_space =
shm_toc_lookup(toc, SESSION_KEY_RECORD_TYPMOD_REGISTRY, false);
SharedRecordTypmodRegistryAttach((SharedRecordTypmodRegistry *)
typmod_registry_space);
+ /* Attach to the shared combo CID registry. */
+ SharedComboCidRegistryAttach();
+
/* Remain attached until end of backend or DetachSession(). */
dsm_pin_mapping(seg);
dsa_pin_mapping(dsa);
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861a02..416ff00510 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2462,11 +2462,6 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
- /*
- * Forbid this during a parallel operation, lest it allocate a combocid.
- * Other workers might need that combocid for visibility checks, and we
- * have no provision for broadcasting it to them.
- */
if (IsInParallelMode())
ereport(ERROR,
(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
@@ -2934,11 +2929,6 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
- /*
- * Forbid this during a parallel operation, lest it allocate a combocid.
- * Other workers might need that combocid for visibility checks, and we
- * have no provision for broadcasting it to them.
- */
if (IsInParallelMode())
ereport(ERROR,
(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
index 99c588d6dc..d78249bd8a 100644
--- a/src/backend/access/transam/README.parallel
+++ b/src/backend/access/transam/README.parallel
@@ -105,12 +105,6 @@ worker. This includes:
checks return the same results in the worker as they do in the
initiating backend. See also the section Transaction Integration, below.
- - The combo CID mappings. This is needed to ensure consistent answers to
- tuple visibility checks. The need to synchronize this data structure is
- a major reason why we can't support writes in parallel mode: such writes
- might create new combo CIDs, and we have no way to let other workers
- (or the initiating backend) know about them.
-
- The transaction snapshot.
- The active snapshot, which might be different from the transaction
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b0426960c7..f643cc28d3 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -204,7 +204,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
MemoryContext oldcontext;
Size library_len = 0;
Size guc_len = 0;
- Size combocidlen = 0;
Size tsnaplen = 0;
Size asnaplen = 0;
Size tstatelen = 0;
@@ -252,8 +251,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
shm_toc_estimate_chunk(&pcxt->estimator, library_len);
guc_len = EstimateGUCStateSpace();
shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
- combocidlen = EstimateComboCIDStateSpace();
- shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
asnaplen = EstimateSnapshotSpace(active_snapshot);
@@ -338,7 +335,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
{
char *libraryspace;
char *gucspace;
- char *combocidspace;
char *tsnapspace;
char *asnapspace;
char *tstatespace;
@@ -361,11 +357,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
SerializeGUCState(guc_len, gucspace);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
- /* Serialize combo CID state. */
- combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
- SerializeComboCIDState(combocidlen, combocidspace);
- shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
-
/* Serialize transaction snapshot and active snapshot. */
tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
SerializeSnapshot(transaction_snapshot, tsnapspace);
@@ -1250,7 +1241,6 @@ ParallelWorkerMain(Datum main_arg)
char *function_name;
parallel_worker_main_type entrypt;
char *gucspace;
- char *combocidspace;
char *tsnapspace;
char *asnapspace;
char *tstatespace;
@@ -1398,10 +1388,6 @@ ParallelWorkerMain(Datum main_arg)
tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE, false);
StartParallelWorkerTransaction(tstatespace);
- /* Restore combo CID state. */
- combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID, false);
- RestoreComboCIDState(combocidspace);
-
/* Attach to the per-session DSM segment and contained objects. */
session_dsm_handle_space =
shm_toc_lookup(toc, PARALLEL_KEY_SESSION_DSM, false);
diff --git a/src/backend/storage/lmgr/lwlock.c b/src/backend/storage/lmgr/lwlock.c
index 2fa90cc095..1a254d1af8 100644
--- a/src/backend/storage/lmgr/lwlock.c
+++ b/src/backend/storage/lmgr/lwlock.c
@@ -170,6 +170,8 @@ static const char *const BuiltinTrancheNames[] = {
"PerSessionRecordType",
/* LWTRANCHE_PER_SESSION_RECORD_TYPMOD: */
"PerSessionRecordTypmod",
+ /* LWTRANCHE_SHARED_COMBOCID: */
+ "SharedComboCid",
/* LWTRANCHE_SHARED_TUPLESTORE: */
"SharedTupleStore",
/* LWTRANCHE_SHARED_TIDBITMAP: */
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index 4ee9ef0ffe..e39a179f76 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -42,6 +42,8 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
+#include "access/session.h"
#include "access/xact.h"
#include "miscadmin.h"
#include "storage/shmem.h"
@@ -61,6 +63,19 @@ typedef struct
typedef ComboCidKeyData *ComboCidKey;
+/*
+ * Shared memory version of the array for use in parallel queries. For now we
+ * don't have a shared memory hash table, we just let each backend deduplicate
+ * as much as it can, but all participating backends can decode each other's
+ * combo CIDs through this structure. It is protected by shared_combocid_lock.
+ */
+typedef struct SharedComboCidRegistry
+{
+ size_t size;
+ size_t used;
+ ComboCidKeyData combocids[FLEXIBLE_ARRAY_MEMBER];
+} SharedComboCidRegistry;
+
typedef struct
{
ComboCidKeyData key;
@@ -69,6 +84,7 @@ typedef struct
typedef ComboCidEntryData *ComboCidEntry;
+
/* Initial size of the hash table */
#define CCID_HASH_SIZE 100
@@ -81,12 +97,19 @@ static ComboCidKey comboCids = NULL;
static int usedComboCids = 0; /* number of elements in comboCids */
static int sizeComboCids = 0; /* allocated size of array */
+/*
+ * For the shared memory version of the above, used for parallel queries, see
+ * session.h.
+ */
+
/* Initial size of the array */
#define CCID_ARRAY_SIZE 100
/* prototypes for internal functions */
static CommandId GetComboCommandId(CommandId cmin, CommandId cmax);
+static CommandId GetSharedComboCommandId(CommandId cmin, CommandId cmax);
+static CommandId GetLocalComboCommandId(CommandId cmin, CommandId cmax);
static CommandId GetRealCmin(CommandId combocid);
static CommandId GetRealCmax(CommandId combocid);
@@ -190,6 +213,15 @@ AtEOXact_ComboCid(void)
comboCids = NULL;
usedComboCids = 0;
sizeComboCids = 0;
+
+ /*
+ * If we're attached a shared registry, the leader marks it empty, but
+ * we'll keep the memory around for use by future transactions.
+ */
+ if (!IsParallelWorker() &&
+ CurrentSession &&
+ CurrentSession->shared_combocid_registry)
+ CurrentSession->shared_combocid_registry->used = 0;
}
@@ -198,16 +230,15 @@ AtEOXact_ComboCid(void)
/*
* Get a combo command id that maps to cmin and cmax.
*
- * We try to reuse old combo command ids when possible.
+ * We try to reuse old combo command ids when possible, but for now we only
+ * consider combos created by this backend. Another process in the same
+ * parallel query could generate a distinct different combo command IDs for the
+ * same transaction, but all processes will be able to understand that combo
+ * command ID.
*/
static CommandId
GetComboCommandId(CommandId cmin, CommandId cmax)
{
- CommandId combocid;
- ComboCidKeyData key;
- ComboCidEntry entry;
- bool found;
-
/*
* Create the hash table and array the first time we need to use combo
* cids in the transaction.
@@ -234,6 +265,99 @@ GetComboCommandId(CommandId cmin, CommandId cmax)
HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
}
+ if (CurrentSession->shared_combocid_registry)
+ return GetSharedComboCommandId(cmin, cmax);
+ else
+ return GetLocalComboCommandId(cmin, cmax);
+}
+
+static CommandId
+GetSharedComboCommandId(CommandId cmin, CommandId cmax)
+{
+ CommandId combocid;
+ ComboCidKeyData key;
+ ComboCidEntry entry;
+ bool found;
+
+ /* Check if we already have it before taking any locks. */
+ key.cmin = cmin;
+ key.cmax = cmax;
+ entry = (ComboCidEntry) hash_search(comboHash,
+ &key,
+ HASH_ENTER,
+ &found);
+ if (found)
+ return entry->combocid;
+
+ /* We'll create a new one in shared memory. */
+ LWLockAcquire(&CurrentSession->fixed->shared_combocid_lock, LW_EXCLUSIVE);
+
+ /* If the shared memory array is already full, we'll have to expand it. */
+ if (CurrentSession->shared_combocid_registry->used ==
+ CurrentSession->shared_combocid_registry->size)
+ {
+ SharedComboCidRegistry *old_data;
+ SharedComboCidRegistry *new_data;
+ dsa_pointer new_data_dsa;
+ size_t new_size;
+
+ /* Double the size of the array. */
+ new_size = CurrentSession->shared_combocid_registry->size * 2;
+ new_data_dsa = dsa_allocate_extended(CurrentSession->area,
+ offsetof(SharedComboCidRegistry,
+ combocids) +
+ sizeof(ComboCidKeyData) *
+ new_size,
+ DSA_ALLOC_NO_OOM);
+ if (new_data_dsa == InvalidDsaPointer)
+ {
+ /* Undo the new hash table entry. */
+ hash_search(comboHash, &key, HASH_REMOVE, &found);
+ LWLockRelease(&CurrentSession->fixed->shared_combocid_lock);
+ elog(ERROR, "out of memory");
+ }
+
+ /* Copy the old contents into the new array. */
+ old_data = CurrentSession->shared_combocid_registry;
+ new_data = (SharedComboCidRegistry *)
+ dsa_get_address(CurrentSession->area, new_data_dsa);
+ memcpy(new_data, old_data, offsetof(SharedComboCidRegistry,
+ combocids) +
+ sizeof(ComboCidKeyData) *
+ old_data->used);
+ new_data->size = new_size;
+
+ /* Free the old array. */
+ dsa_free(CurrentSession->area,
+ CurrentSession->fixed->shared_combocid_registry_dsa);
+
+ /* Advertise the new array for other backends to notice. */
+ CurrentSession->fixed->shared_combocid_registry_dsa = new_data_dsa;
+ CurrentSession->fixed->shared_combocid_change++;
+
+ /* Set our own local pointer so we can access it. */
+ CurrentSession->shared_combocid_registry = new_data;
+ }
+
+ /* Now we can add a new entry. */
+ combocid = CurrentSession->shared_combocid_registry->used++;
+ CurrentSession->shared_combocid_registry->combocids[combocid].cmin = cmin;
+ CurrentSession->shared_combocid_registry->combocids[combocid].cmax = cmax;
+ entry->combocid = combocid;
+
+ LWLockRelease(&CurrentSession->fixed->shared_combocid_lock);
+
+ return combocid;
+}
+
+static CommandId
+GetLocalComboCommandId(CommandId cmin, CommandId cmax)
+{
+ CommandId combocid;
+ ComboCidKeyData key;
+ ComboCidEntry entry;
+ bool found;
+
/*
* Grow the array if there's not at least one free slot. We must do this
* before possibly entering a new hashtable entry, else failure to
@@ -276,90 +400,118 @@ GetComboCommandId(CommandId cmin, CommandId cmax)
return combocid;
}
+/*
+ * Another backend could have replaced the array in order to expand it. Make
+ * sure that CurrentSession->shared_combocid_registry points to the current
+ * one.
+ */
+static inline void
+ensure_shared_combocid_registry(void)
+{
+ if (unlikely(CurrentSession->fixed->shared_combocid_change !=
+ CurrentSession->shared_combocid_change))
+ {
+ CurrentSession->shared_combocid_registry = (SharedComboCidRegistry *)
+ dsa_get_address(CurrentSession->area,
+ CurrentSession->fixed->shared_combocid_registry_dsa);
+ CurrentSession->shared_combocid_change =
+ CurrentSession->fixed->shared_combocid_change;
+ }
+}
+
static CommandId
GetRealCmin(CommandId combocid)
{
- Assert(combocid < usedComboCids);
- return comboCids[combocid].cmin;
+ if (CurrentSession->shared_combocid_registry)
+ {
+ CommandId result;
+
+ /*
+ * XXX Use local comboCids array as a cache, to avoid acquiring the
+ * lock?
+ */
+
+ LWLockAcquire(&CurrentSession->fixed->shared_combocid_lock, LW_SHARED);
+ ensure_shared_combocid_registry();
+ result = CurrentSession->shared_combocid_registry->combocids[combocid].cmin;
+ LWLockRelease(&CurrentSession->fixed->shared_combocid_lock);
+
+ return result;
+
+ }
+ else
+ {
+ Assert(combocid < usedComboCids);
+ return comboCids[combocid].cmin;
+ }
}
static CommandId
GetRealCmax(CommandId combocid)
{
- Assert(combocid < usedComboCids);
- return comboCids[combocid].cmax;
+ if (CurrentSession->shared_combocid_registry)
+ {
+ CommandId result;
+
+ LWLockAcquire(&CurrentSession->fixed->shared_combocid_lock, LW_SHARED);
+ ensure_shared_combocid_registry();
+ result = CurrentSession->shared_combocid_registry->combocids[combocid].cmax;
+ LWLockRelease(&CurrentSession->fixed->shared_combocid_lock);
+
+ return result;
+ }
+ else
+ {
+ Assert(combocid < usedComboCids);
+ return comboCids[combocid].cmax;
+ }
}
-/*
- * Estimate the amount of space required to serialize the current ComboCID
- * state.
- */
-Size
-EstimateComboCIDStateSpace(void)
+void
+SharedComboCidRegistryInit(dsm_segment *seg, dsa_area *area)
{
- Size size;
+ SharedComboCidRegistry *new_data;
+ dsa_pointer new_data_dsa;
+ size_t new_size;
- /* Add space required for saving usedComboCids */
- size = sizeof(int);
+ /*
+ * No need to acquire the lock, because during initialization no workers
+ * are running yet.
+ */
- /* Add space required for saving the combocids key */
- size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+ new_size = Min(usedComboCids, 16);
- return size;
-}
+ new_data_dsa = dsa_allocate(area,
+ offsetof(SharedComboCidRegistry,
+ combocids) +
+ sizeof(ComboCidKeyData) * new_size);
+ new_data = (SharedComboCidRegistry *) dsa_get_address(area, new_data_dsa);
-/*
- * Serialize the ComboCID state into the memory, beginning at start_address.
- * maxsize should be at least as large as the value returned by
- * EstimateComboCIDStateSpace.
- */
-void
-SerializeComboCIDState(Size maxsize, char *start_address)
-{
- char *endptr;
+ /*
+ * Copy all existing combos into shared memory. From now on this session
+ * will be using shared memory for this.
+ */
+ new_data->size = new_size;
+ new_data->used = usedComboCids;
+ memcpy(&new_data->combocids, comboCids,
+ sizeof(ComboCidKeyData) * usedComboCids);
- /* First, we store the number of currently-existing ComboCIDs. */
- *(int *) start_address = usedComboCids;
+ /* Advertise the new array for other backends to notice. */
+ CurrentSession->fixed->shared_combocid_registry_dsa = new_data_dsa;
+ CurrentSession->fixed->shared_combocid_change = 1;
- /* If maxsize is too small, throw an error. */
- endptr = start_address + sizeof(int) +
- (sizeof(ComboCidKeyData) * usedComboCids);
- if (endptr < start_address || endptr > start_address + maxsize)
- elog(ERROR, "not enough space to serialize ComboCID state");
+ /* Set our own local pointer so we can access it. */
+ CurrentSession->shared_combocid_registry = new_data;
- /* Now, copy the actual cmin/cmax pairs. */
- if (usedComboCids > 0)
- memcpy(start_address + sizeof(int), comboCids,
- (sizeof(ComboCidKeyData) * usedComboCids));
+ /* XXX install cleanup callback? */
}
-/*
- * Read the ComboCID state at the specified address and initialize this
- * backend with the same ComboCIDs. This is only valid in a backend that
- * currently has no ComboCIDs (and only makes sense if the transaction state
- * is serialized and restored as well).
- */
void
-RestoreComboCIDState(char *comboCIDstate)
+SharedComboCidRegistryAttach(void)
{
- int num_elements;
- ComboCidKeyData *keydata;
- int i;
- CommandId cid;
+ LWLockAcquire(&CurrentSession->fixed->shared_combocid_lock, LW_SHARED);
+ ensure_shared_combocid_registry();
+ LWLockRelease(&CurrentSession->fixed->shared_combocid_lock);
- Assert(!comboCids && !comboHash);
-
- /* First, we retrieve the number of ComboCIDs that were serialized. */
- num_elements = *(int *) comboCIDstate;
- keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
-
- /* Use GetComboCommandId to restore each ComboCID. */
- for (i = 0; i < num_elements; i++)
- {
- cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
-
- /* Verify that we got the expected answer. */
- if (cid != i)
- elog(ERROR, "unexpected command ID while restoring combo CIDs");
- }
+ /* XXX install cleanup callback? */
}
diff --git a/src/include/access/session.h b/src/include/access/session.h
index 4c1f6ffd40..9d42546098 100644
--- a/src/include/access/session.h
+++ b/src/include/access/session.h
@@ -13,10 +13,22 @@
#define SESSION_H
#include "lib/dshash.h"
+#include "storage/lwlock.h"
-/* Avoid including typcache.h */
+struct SharedComboCidLock;
struct SharedRecordTypmodRegistry;
+/*
+ * Part of the session object that is of fixed size in shared memory.
+ */
+typedef struct SessionFixed
+{
+ /* State managed by combocid.c. */
+ LWLock shared_combocid_lock;
+ dsa_pointer shared_combocid_registry_dsa;
+ int shared_combocid_change;
+} SessionFixed;
+
/*
* A struct encapsulating some elements of a user's session. For now this
* manages state that applies to parallel query, but in principle it could
@@ -27,6 +39,12 @@ typedef struct Session
dsm_segment *segment; /* The session-scoped DSM segment. */
dsa_area *area; /* The session-scoped DSA area. */
+ SessionFixed *fixed;
+
+ /* State managed by combocid.c. */
+ struct SharedComboCidRegistry *shared_combocid_registry;
+ int shared_combocid_change;
+
/* State managed by typcache.c. */
struct SharedRecordTypmodRegistry *shared_typmod_registry;
dshash_table *shared_record_table;
diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h
index af9b41795d..780eb44b5b 100644
--- a/src/include/storage/lwlock.h
+++ b/src/include/storage/lwlock.h
@@ -214,6 +214,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_PER_SESSION_DSA,
LWTRANCHE_PER_SESSION_RECORD_TYPE,
LWTRANCHE_PER_SESSION_RECORD_TYPMOD,
+ LWTRANCHE_SHARED_COMBOCID,
LWTRANCHE_SHARED_TUPLESTORE,
LWTRANCHE_SHARED_TIDBITMAP,
LWTRANCHE_PARALLEL_APPEND,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index b39166a685..b13e44e1fc 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -14,6 +14,9 @@
#ifndef COMBOCID_H
#define COMBOCID_H
+#include "storage/dsm.h"
+#include "utils/dsa.h"
+
/*
* HeapTupleHeaderGetCmin and HeapTupleHeaderGetCmax function prototypes
* are in access/htup.h, because that's where the macro definitions that
@@ -21,8 +24,7 @@
*/
extern void AtEOXact_ComboCid(void);
-extern void RestoreComboCIDState(char *comboCIDstate);
-extern void SerializeComboCIDState(Size maxsize, char *start_address);
-extern Size EstimateComboCIDStateSpace(void);
+extern void SharedComboCidRegistryInit(dsm_segment *seg, dsa_area *area);
+extern void SharedComboCidRegistryAttach(void);
#endif /* COMBOCID_H */
--
2.20.1
On Fri, Oct 9, 2020 at 8:41 AM Thomas Munro <thomas.munro@gmail.com> wrote:
One thing I noticed is that you have logic, variable names and
assertions all over the tree that assume that we can only do parallel
*inserts*. I agree 100% with your plan to make Parallel Insert work
first, it is an excellent goal and if we get it in it'll be a headline
feature of PG14 (along with COPY etc). That said, I wonder if it
would make sense to use more general naming (isParallelModifyLeader?),
be more liberal where you really mean "is it DML", and find a way to
centralise the logic about which DML commands types are currently
allowed (ie insert only for now) for assertions and error checks etc,
so that in future we don't have to go around and change all these
places and rename things again and again.
Fair points.
I agree, it would make more sense to generalise the naming and
centralise the DML-command-type checks, rather than everything being
insert-specific.
It was getting a bit ugly. I'll work on that.
While contemplating that, I couldn't resist taking a swing at the main
(?) show stopper for Parallel Update and Parallel Delete, judging by
various clues left in code comments by Robert: combo command IDs
created by other processes. Here's a rapid prototype to make that
work (though perhaps not as efficiently as we'd want, not sure). With
that in place, I wonder what else we'd need to extend your patch to
cover all three operations... it can't be much! Of course I don't
want to derail your work on Parallel Insert, I'm just providing some
motivation for my comments on the (IMHO) shortsightedness of some of
the coding.
Thanks for your prototype code for coordination of combo command IDs
with the workers.
It does give me the incentive to look beyond that issue and see
whether parallel Update and parallel Delete are indeed possible. I'll
be sure to give it a go!
PS Why not use git format-patch to create patches?
Guess I was being a bit lazy - will use git format-patch in future.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 9, 2020 at 3:48 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
It does give me the incentive to look beyond that issue and see
whether parallel Update and parallel Delete are indeed possible. I'll
be sure to give it a go!
Cool!
A couple more observations:
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
Hmm, I think this may be bogus window dressing only affecting EXPLAIN.
If you change it to assign false always, it works just the same,
except EXPLAIN says:
Gather (cost=15428.00..16101.14 rows=1000000 width=4)
Workers Planned: 2
-> Insert on s (cost=15428.00..16101.14 rows=208334 width=4)
-> Parallel Hash Join (cost=15428.00..32202.28 rows=416667 width=4)
... instead of:
Gather (cost=15428.00..16101.14 rows=1000000 width=4)
Workers Planned: 2
-> Parallel Insert on s (cost=15428.00..16101.14 rows=208334 width=4)
-> Parallel Hash Join (cost=15428.00..32202.28 rows=416667 width=4)
AFAICS it's not parallel-aware, it just happens to be running in
parallel with a partial input and partial output (and in this case,
effect in terms of writes). Parallel-aware is our term for nodes that
actually know they are running in parallel and do some special
coordination with their twins in other processes.
The estimated row count also looks wrong; at a guess, the parallel
divisor is applied twice. Let me try that with
parallel_leader_particiation=off (which disables some funky maths in
the row estimation and makes it straight division by number of
processes):
Gather (cost=17629.00..18645.50 rows=1000000 width=4)
Workers Planned: 2
-> Insert on s (cost=17629.00..18645.50 rows=250000 width=4)
-> Parallel Hash Join (cost=17629.00..37291.00 rows=500000 width=4)
[more nodes omitted]
Yeah, that was a join that spat out a million rows, and we correctly
estimated 500k per process, and then Insert (still with my hack to
turn off the bogus "Parallel" display in this case, but it doesn't
affect the estimation) estimated 250k per process, which is wrong.
On Fri, Oct 9, 2020 at 6:31 PM Thomas Munro <thomas.munro@gmail.com> wrote:
A couple more observations:
+ pathnode->path.parallel_aware = parallel_workers > 0 ? true : false;
Hmm, I think this may be bogus window dressing only affecting EXPLAIN.
If you change it to assign false always, it works just the same,
except EXPLAIN says:Gather (cost=15428.00..16101.14 rows=1000000 width=4)
Workers Planned: 2
-> Insert on s (cost=15428.00..16101.14 rows=208334 width=4)
-> Parallel Hash Join (cost=15428.00..32202.28 rows=416667 width=4)... instead of:
Gather (cost=15428.00..16101.14 rows=1000000 width=4)
Workers Planned: 2
-> Parallel Insert on s (cost=15428.00..16101.14 rows=208334 width=4)
-> Parallel Hash Join (cost=15428.00..32202.28 rows=416667 width=4)AFAICS it's not parallel-aware, it just happens to be running in
parallel with a partial input and partial output (and in this case,
effect in terms of writes). Parallel-aware is our term for nodes that
actually know they are running in parallel and do some special
coordination with their twins in other processes.
Ah, thanks, I see the distinction now. I'll fix that, to restore
parallel_aware=false for the ModifyTable node.
The estimated row count also looks wrong; at a guess, the parallel
divisor is applied twice. Let me try that with
parallel_leader_particiation=off (which disables some funky maths in
the row estimation and makes it straight division by number of
processes):Gather (cost=17629.00..18645.50 rows=1000000 width=4)
Workers Planned: 2
-> Insert on s (cost=17629.00..18645.50 rows=250000 width=4)
-> Parallel Hash Join (cost=17629.00..37291.00 rows=500000 width=4)
[more nodes omitted]Yeah, that was a join that spat out a million rows, and we correctly
estimated 500k per process, and then Insert (still with my hack to
turn off the bogus "Parallel" display in this case, but it doesn't
affect the estimation) estimated 250k per process, which is wrong.
Thanks, I did suspect the current costing was wrong for ModifyTable
(workers>0 case), as I'd thrown it in (moving current costing code
into costsize.c) without a lot of checking or great thought, and was
on my TODO list of things to check. At least I created a placeholder
for it. Looks like I've applied a parallel-divisor again (not allowing
for that of the underlying query), as you said.
Speaking of costing, I'm not sure I really agree with the current
costing of a Gather node. Just considering a simple Parallel SeqScan
case, the "run_cost += parallel_tuple_cost * path->path.rows;" part of
Gather cost always completely drowns out any other path costs when a
large number of rows are involved (at least with default
parallel-related GUC values), such that Parallel SeqScan would never
be the cheapest path. This linear relationship in the costing based on
the rows and a parallel_tuple_cost doesn't make sense to me. Surely
after a certain amount of rows, the overhead of launching workers will
be out-weighed by the benefit of their parallel work, such that the
more rows, the more likely a Parallel SeqScan will benefit. That seems
to suggest something like a logarithmic formula (or similar) would
better match reality than what we have now. Am I wrong on this? Every
time I use default GUC values, the planner doesn't want to generate a
parallel plan. Lowering parallel-related GUCs like parallel_tuple_cost
(which I normally do for testing) influences it of course, but the
linear relationship still seems wrong.
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Oct 6, 2020 at 3:08 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Oct 5, 2020 at 10:36 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Also, I have attached a separate patch (requested by Andres Freund)
that just allows the underlying SELECT part of "INSERT INTO ... SELECT
..." to be parallel.
It might be a good idea to first just get this patch committed, if
possible. So, I have reviewed the latest version of this patch:
0001-InsertParallelSelect
1.
ParallelContext *pcxt;
+ /*
+ * We need to avoid an attempt on INSERT to assign a
+ * FullTransactionId whilst in parallel mode (which is in
+ * effect due to the underlying parallel query) - so the
+ * FullTransactionId is assigned here. Parallel mode must
+ * be temporarily escaped in order for this to be possible.
+ * The FullTransactionId will be included in the transaction
+ * state that is serialized in the parallel DSM.
+ */
+ if (estate->es_plannedstmt->commandType == CMD_INSERT)
+ {
+ Assert(IsInParallelMode());
+ ExitParallelMode();
+ GetCurrentFullTransactionId();
+ EnterParallelMode();
+ }
+
This looks like a hack to me. I think you are doing this to avoid the
parallel mode checks in GetNewTransactionId(), right? If so, I have
already mentioned above [1]/messages/by-id/CAA4eK1+E-pM0U6qw7EOF0yO0giTxdErxoJV9xTqN+Lo9zdotFQ@mail.gmail.com that we can change it so that we disallow
assigning xids for parallel workers only. The same is true for the
check in ExecGatherMerge. Do you see any problem with that suggestion?
2.
@@ -337,7 +337,7 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
I think the comments above this need to be updated especially the part
where we says:"Note that we do allow CREATE TABLE AS, SELECT INTO, and
CREATE MATERIALIZED VIEW to use parallel plans, but as of now, only
the leader backend writes into a completely new table.". Don't we need
to include Insert also?
3.
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);
Why do you need this change? The comments above this code should be
updated to reflect this change. I think for the same reason the below
code seems to be modified but I don't understand the reason for the
below change as well, also it is better to update the comments for
this as well.
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
* Optionally add a Gather node for testing purposes, provided this is
* actually a safe thing to do.
*/
- if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe)
+ if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType
== CMD_SELECT && top_plan->parallel_safe)
{
Gather *gather = makeNode(Gather);
[1]: /messages/by-id/CAA4eK1+E-pM0U6qw7EOF0yO0giTxdErxoJV9xTqN+Lo9zdotFQ@mail.gmail.com
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 2:37 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Speaking of costing, I'm not sure I really agree with the current
costing of a Gather node. Just considering a simple Parallel SeqScan
case, the "run_cost += parallel_tuple_cost * path->path.rows;" part of
Gather cost always completely drowns out any other path costs when a
large number of rows are involved (at least with default
parallel-related GUC values), such that Parallel SeqScan would never
be the cheapest path. This linear relationship in the costing based on
the rows and a parallel_tuple_cost doesn't make sense to me. Surely
after a certain amount of rows, the overhead of launching workers will
be out-weighed by the benefit of their parallel work, such that the
more rows, the more likely a Parallel SeqScan will benefit.
That will be true for the number of rows/pages we need to scan not for
the number of tuples we need to return as a result. The formula here
considers the number of rows the parallel scan will return and the
more the number of rows each parallel node needs to pass via shared
memory to gather node the more costly it will be.
We do consider the total pages we need to scan in
compute_parallel_worker() where we use a logarithmic formula to
determine the number of workers.
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
0001-InsertParallelSelect
1.
ParallelContext *pcxt;+ /* + * We need to avoid an attempt on INSERT to assign a + * FullTransactionId whilst in parallel mode (which is in + * effect due to the underlying parallel query) - so the + * FullTransactionId is assigned here. Parallel mode must + * be temporarily escaped in order for this to be possible. + * The FullTransactionId will be included in the transaction + * state that is serialized in the parallel DSM. + */ + if (estate->es_plannedstmt->commandType == CMD_INSERT) + { + Assert(IsInParallelMode()); + ExitParallelMode(); + GetCurrentFullTransactionId(); + EnterParallelMode(); + } +This looks like a hack to me. I think you are doing this to avoid the
parallel mode checks in GetNewTransactionId(), right?
Yes, agreed, is a hack to avoid that (mind you, it's not exactly great
that ExecutePlan() sets parallel-mode for the entire plan execution).
Also, did not expect that to necessarily remain in a final patch.
If so, I have
already mentioned above [1] that we can change it so that we disallow
assigning xids for parallel workers only. The same is true for the
check in ExecGatherMerge. Do you see any problem with that suggestion?
No, should be OK I guess, but will update and test to be sure.
2. @@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions, */ if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster && - parse->commandType == CMD_SELECT && + (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) && !parse->hasModifyingCTE && max_parallel_workers_per_gather > 0 && !IsParallelWorker())I think the comments above this need to be updated especially the part
where we says:"Note that we do allow CREATE TABLE AS, SELECT INTO, and
CREATE MATERIALIZED VIEW to use parallel plans, but as of now, only
the leader backend writes into a completely new table.". Don't we need
to include Insert also?
Yes, Insert needs to be mentioned somewhere there.
3.
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);Why do you need this change? The comments above this code should be
updated to reflect this change. I think for the same reason the below
code seems to be modified but I don't understand the reason for the
below change as well, also it is better to update the comments for
this as well.
OK, I will update the comments for this.
Basically, up to now, the "force_parallel_mode" has only ever operated
on a SELECT.
But since we are now allowing CMD_INSERT to be assessed for parallel
mode too, we need to prevent the force_parallel_mode logic from
sticking a Gather node over the top of arbitrary INSERTs and causing
them to be run in parallel. Not all INSERTs are suitable for parallel
operation, and also there are further considerations for
parallel-safety for INSERTs compared to SELECT. INSERTs can also
trigger UPDATEs.
If we need to support force_parallel_mode for INSERT, more work will
need to be done.
@@ -425,7 +426,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions, * Optionally add a Gather node for testing purposes, provided this is * actually a safe thing to do. */ - if (force_parallel_mode != FORCE_PARALLEL_OFF && top_plan->parallel_safe) + if (force_parallel_mode != FORCE_PARALLEL_OFF && parse->commandType == CMD_SELECT && top_plan->parallel_safe) { Gather *gather = makeNode(Gather);[1] - /messages/by-id/CAA4eK1+E-pM0U6qw7EOF0yO0giTxdErxoJV9xTqN+Lo9zdotFQ@mail.gmail.com
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 9, 2020 at 8:41 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Oct 9, 2020 at 2:37 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Speaking of costing, I'm not sure I really agree with the current
costing of a Gather node. Just considering a simple Parallel SeqScan
case, the "run_cost += parallel_tuple_cost * path->path.rows;" part of
Gather cost always completely drowns out any other path costs when a
large number of rows are involved (at least with default
parallel-related GUC values), such that Parallel SeqScan would never
be the cheapest path. This linear relationship in the costing based on
the rows and a parallel_tuple_cost doesn't make sense to me. Surely
after a certain amount of rows, the overhead of launching workers will
be out-weighed by the benefit of their parallel work, such that the
more rows, the more likely a Parallel SeqScan will benefit.That will be true for the number of rows/pages we need to scan not for
the number of tuples we need to return as a result. The formula here
considers the number of rows the parallel scan will return and the
more the number of rows each parallel node needs to pass via shared
memory to gather node the more costly it will be.We do consider the total pages we need to scan in
compute_parallel_worker() where we use a logarithmic formula to
determine the number of workers.
Despite all the best intentions, the current costings seem to be
geared towards selection of a non-parallel plan over a parallel plan,
the more rows there are in the table. Yet the performance of a
parallel plan appears to be better than non-parallel-plan the more
rows there are in the table.
This doesn't seem right to me. Is there a rationale behind this costing model?
I have pointed out the part of the parallel_tuple_cost calculation
that seems to drown out all other costs (causing the cost value to be
huge), the more rows there are in the table.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 9, 2020 at 4:28 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:41 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Oct 9, 2020 at 2:37 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Speaking of costing, I'm not sure I really agree with the current
costing of a Gather node. Just considering a simple Parallel SeqScan
case, the "run_cost += parallel_tuple_cost * path->path.rows;" part of
Gather cost always completely drowns out any other path costs when a
large number of rows are involved (at least with default
parallel-related GUC values), such that Parallel SeqScan would never
be the cheapest path. This linear relationship in the costing based on
the rows and a parallel_tuple_cost doesn't make sense to me. Surely
after a certain amount of rows, the overhead of launching workers will
be out-weighed by the benefit of their parallel work, such that the
more rows, the more likely a Parallel SeqScan will benefit.That will be true for the number of rows/pages we need to scan not for
the number of tuples we need to return as a result. The formula here
considers the number of rows the parallel scan will return and the
more the number of rows each parallel node needs to pass via shared
memory to gather node the more costly it will be.We do consider the total pages we need to scan in
compute_parallel_worker() where we use a logarithmic formula to
determine the number of workers.Despite all the best intentions, the current costings seem to be
geared towards selection of a non-parallel plan over a parallel plan,
the more rows there are in the table. Yet the performance of a
parallel plan appears to be better than non-parallel-plan the more
rows there are in the table.
This doesn't seem right to me. Is there a rationale behind this costing model?
Yes, AFAIK, there is no proof that we can get any (much) gain by
dividing the I/O among workers. It is primarily the CPU effort which
gives the benefit. So, the parallel plans show greater benefit when we
have to scan a large table and then project much lesser rows.
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
+ /* + * We need to avoid an attempt on INSERT to assign a + * FullTransactionId whilst in parallel mode (which is in + * effect due to the underlying parallel query) - so the + * FullTransactionId is assigned here. Parallel mode must + * be temporarily escaped in order for this to be possible. + * The FullTransactionId will be included in the transaction + * state that is serialized in the parallel DSM. + */ + if (estate->es_plannedstmt->commandType == CMD_INSERT) + { + Assert(IsInParallelMode()); + ExitParallelMode(); + GetCurrentFullTransactionId(); + EnterParallelMode(); + } +This looks like a hack to me. I think you are doing this to avoid the
parallel mode checks in GetNewTransactionId(), right? If so, I have
already mentioned above [1] that we can change it so that we disallow
assigning xids for parallel workers only. The same is true for the
check in ExecGatherMerge. Do you see any problem with that suggestion?
Actually, there is a problem.
If I remove that "hack", and change the code in GetNewTransactionId()
to disallow xid assignment for parallel workers only, then there is
also similar code in AssignTransactionId() which gets called. If I
change that code too, in the same way, then on a parallel INSERT, that
code gets called by a parallel worker (from GetCurrentTransactionId())
and the ERROR "cannot assign XIDs in a parallel worker" results.
GetCurrentFullTransactionId() must be called in the leader, somewhere
(and will be included in the transaction state that is serialized in
the parallel DSM).
If not done here, then where?
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 9, 2020 at 5:54 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
+ /* + * We need to avoid an attempt on INSERT to assign a + * FullTransactionId whilst in parallel mode (which is in + * effect due to the underlying parallel query) - so the + * FullTransactionId is assigned here. Parallel mode must + * be temporarily escaped in order for this to be possible. + * The FullTransactionId will be included in the transaction + * state that is serialized in the parallel DSM. + */ + if (estate->es_plannedstmt->commandType == CMD_INSERT) + { + Assert(IsInParallelMode()); + ExitParallelMode(); + GetCurrentFullTransactionId(); + EnterParallelMode(); + } +This looks like a hack to me. I think you are doing this to avoid the
parallel mode checks in GetNewTransactionId(), right? If so, I have
already mentioned above [1] that we can change it so that we disallow
assigning xids for parallel workers only. The same is true for the
check in ExecGatherMerge. Do you see any problem with that suggestion?Actually, there is a problem.
If I remove that "hack", and change the code in GetNewTransactionId()
to disallow xid assignment for parallel workers only, then there is
also similar code in AssignTransactionId() which gets called.
I don't think workers need to call AssignTransactionId(), before that
the transactionid passed from leader should be set in
CurrentTransactionState. Why
GetCurrentTransactionId()/GetCurrentFullTransactionId(void) needs to
call AssignTransactionId() when called from worker?
GetCurrentFullTransactionId() must be called in the leader, somewhere
(and will be included in the transaction state that is serialized in
the parallel DSM).
Yes, it should have done in the leader and then it should have been
set in the workers via StartParallelWorkerTransaction before we do any
actual operation. If that happens then GetCurrentTransactionId() won't
need to call AssignTransactionId().
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 3:51 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
0001-InsertParallelSelect
1.
ParallelContext *pcxt;+ /* + * We need to avoid an attempt on INSERT to assign a + * FullTransactionId whilst in parallel mode (which is in + * effect due to the underlying parallel query) - so the + * FullTransactionId is assigned here. Parallel mode must + * be temporarily escaped in order for this to be possible. + * The FullTransactionId will be included in the transaction + * state that is serialized in the parallel DSM. + */ + if (estate->es_plannedstmt->commandType == CMD_INSERT) + { + Assert(IsInParallelMode()); + ExitParallelMode(); + GetCurrentFullTransactionId(); + EnterParallelMode(); + } +This looks like a hack to me. I think you are doing this to avoid the
parallel mode checks in GetNewTransactionId(), right?Yes, agreed, is a hack to avoid that (mind you, it's not exactly great
that ExecutePlan() sets parallel-mode for the entire plan execution).
Also, did not expect that to necessarily remain in a final patch.If so, I have
already mentioned above [1] that we can change it so that we disallow
assigning xids for parallel workers only. The same is true for the
check in ExecGatherMerge. Do you see any problem with that suggestion?No, should be OK I guess, but will update and test to be sure.
2. @@ -337,7 +337,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions, */ if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster && - parse->commandType == CMD_SELECT && + (parse->commandType == CMD_SELECT || parse->commandType == CMD_INSERT) && !parse->hasModifyingCTE && max_parallel_workers_per_gather > 0 && !IsParallelWorker())I think the comments above this need to be updated especially the part
where we says:"Note that we do allow CREATE TABLE AS, SELECT INTO, and
CREATE MATERIALIZED VIEW to use parallel plans, but as of now, only
the leader backend writes into a completely new table.". Don't we need
to include Insert also?Yes, Insert needs to be mentioned somewhere there.
3.
@@ -371,6 +371,7 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
* parallel-unsafe, or else the query planner itself has a bug.
*/
glob->parallelModeNeeded = glob->parallelModeOK &&
+ (parse->commandType == CMD_SELECT) &&
(force_parallel_mode != FORCE_PARALLEL_OFF);Why do you need this change? The comments above this code should be
updated to reflect this change. I think for the same reason the below
code seems to be modified but I don't understand the reason for the
below change as well, also it is better to update the comments for
this as well.OK, I will update the comments for this.
Basically, up to now, the "force_parallel_mode" has only ever operated
on a SELECT.
But since we are now allowing CMD_INSERT to be assessed for parallel
mode too, we need to prevent the force_parallel_mode logic from
sticking a Gather node over the top of arbitrary INSERTs and causing
them to be run in parallel. Not all INSERTs are suitable for parallel
operation, and also there are further considerations for
parallel-safety for INSERTs compared to SELECT. INSERTs can also
trigger UPDATEs.
Sure but in that case 'top_plan->parallel_safe' should be false and it
should stick Gather node atop Insert node. For the purpose of this
patch, the scan beneath Insert should be considered as parallel_safe.
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 6:26 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Oct 9, 2020 at 3:51 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
OK, I will update the comments for this.
Basically, up to now, the "force_parallel_mode" has only ever operated
on a SELECT.
But since we are now allowing CMD_INSERT to be assessed for parallel
mode too, we need to prevent the force_parallel_mode logic from
sticking a Gather node over the top of arbitrary INSERTs and causing
them to be run in parallel. Not all INSERTs are suitable for parallel
operation, and also there are further considerations for
parallel-safety for INSERTs compared to SELECT. INSERTs can also
trigger UPDATEs.Sure but in that case 'top_plan->parallel_safe' should be false and it
should stick Gather node atop Insert node.
/should/should not.
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 11:57 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Sure but in that case 'top_plan->parallel_safe' should be false and it
should stick Gather node atop Insert node./should/should not.
OK, for the minimal patch, just allowing INSERT with parallel SELECT,
you're right, neither of those additional "commandType == CMD_SELECT"
checks are needed, so I'll remove them. (In the main patch, I think
the first check can be removed, once the XID handling is fixed; the
second check is definitely needed though).
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 9, 2020 at 11:58 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:41 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
That will be true for the number of rows/pages we need to scan not for
the number of tuples we need to return as a result. The formula here
considers the number of rows the parallel scan will return and the
more the number of rows each parallel node needs to pass via shared
memory to gather node the more costly it will be.We do consider the total pages we need to scan in
compute_parallel_worker() where we use a logarithmic formula to
determine the number of workers.Despite all the best intentions, the current costings seem to be
geared towards selection of a non-parallel plan over a parallel plan,
the more rows there are in the table. Yet the performance of a
parallel plan appears to be better than non-parallel-plan the more
rows there are in the table.
Right, but as Amit said, we still have to account for the cost of
schlepping tuples between processes. Hmm... could the problem be that
we're incorrectly estimating that Insert (without RETURNING) will send
a bazillion tuples, even though that isn't true? I didn't look at the
code but that's what the plan seems to imply when it says stuff like
"Gather (cost=15428.00..16101.14 rows=1000000 width=4)". I suppose
the row estimates for ModifyTable paths are based on what they write,
not what they emit, and in the past that distinction didn't matter
much because it wasn't something that was used for comparing
alternative plans. Now it is.
On Fri, Oct 9, 2020 at 7:32 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 11:57 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Sure but in that case 'top_plan->parallel_safe' should be false and it
should stick Gather node atop Insert node./should/should not.
OK, for the minimal patch, just allowing INSERT with parallel SELECT,
you're right, neither of those additional "commandType == CMD_SELECT"
checks are needed, so I'll remove them.
Okay, that makes sense.
(In the main patch, I think
the first check can be removed, once the XID handling is fixed; the
second check is definitely needed though).
Okay, then move that check but please do add some comments to state the reason.
--
With Regards,
Amit Kapila.
On Fri, Oct 9, 2020 at 2:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Oct 6, 2020 at 3:08 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Oct 5, 2020 at 10:36 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Also, I have attached a separate patch (requested by Andres Freund)
that just allows the underlying SELECT part of "INSERT INTO ... SELECT
..." to be parallel.It might be a good idea to first just get this patch committed, if
possible. So, I have reviewed the latest version of this patch:
Few initial comments on 0004-ParallelInsertSelect:
1.
@@ -2049,11 +2049,6 @@ heap_prepare_insert(Relation relation,
HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
-
I have speculated above [1]/messages/by-id/CAA4eK1KyftVDgovvRQmdV1b=nN0R-KqdWZqiu7jZ1GYQ7SO9OA@mail.gmail.com to see if we can change this Assert
condition instead of just removing it? Have you considered that
suggestion?
2.
@@ -764,12 +778,13 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed
+ * if currentCommandIdUsed was already true at the start of the
+ * parallel operation (by way of SetCurrentCommandIdUsed()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision
+ * for communicating this back to the leader.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
Once we allowed this, won't the next CommandCounterIncrement() in the
worker will increment the commandId which will lead to using different
commandIds in worker and leader? Is that prevented in some way, if so,
how? Can we document the same?
3.
@@ -173,7 +173,7 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* against performing unsafe operations in parallel mode, but this gives a
* more user-friendly error message.
*/
- if ((XactReadOnly || IsInParallelMode()) &&
+ if ((XactReadOnly || (IsInParallelMode() &&
queryDesc->plannedstmt->commandType != CMD_INSERT)) &&
!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
I don't think above change is correct. We need to extend the below
check in ExecCheckXactReadOnly() because otherwise, it can allow
Insert operations even when XactReadOnly is set which we don't want.
ExecCheckXactReadOnly()
{
..
if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
..
}
4.
@@ -173,18 +175,20 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
- pstmt->hasReturning = false;
- pstmt->hasModifyingCTE = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
+ pstmt->hasModifyingCTE = estate->es_plannedstmt->hasModifyingCTE;
Why change hasModifyingCTE?
5.
+ if (isParallelInsertLeader)
+ {
+ /* For Parallel INSERT, if there are BEFORE STATEMENT triggers,
+ * these must be fired by the leader, not the parallel workers.
+ */
The multi-line comment should start from the second line. I see a
similar problem at other places in the patch as well.
6.
@@ -178,6 +214,25 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelInsertLeader)
+ {
+ /* For Parallel INSERT, if there are BEFORE STATEMENT triggers,
+ * these must be fired by the leader, not the parallel workers.
+ */
+ if (nodeModifyTableState->fireBSTriggers)
+ {
+ fireBSTriggers(nodeModifyTableState);
+ nodeModifyTableState->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local
+ * plan execution (ModifyTable processing). These will be
+ * fired at end of Gather processing.
+ */
+ nodeModifyTableState->fireASTriggers = false;
+ }
+ }
Can we encapsulate this in a separate function? It seems a bit odd to
directly do this ExecGather.
7.
@@ -418,14 +476,25 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
So after this patch if "node->pei == NULL" then we won't shutdown
workers here? Why so?
8. You have made changes related to trigger execution for Gather node,
don't we need similar changes for GatherMerge node?
9.
@@ -383,7 +444,21 @@ cost_gather(GatherPath *path, PlannerInfo *root,
/* Parallel setup and communication cost. */
startup_cost += parallel_setup_cost;
- run_cost += parallel_tuple_cost * path->path.rows;
+
+ /*
+ * For Parallel INSERT, provided no tuples are returned from workers
+ * to gather/leader node, don't add a cost-per-row, as each worker
+ * parallelly inserts the tuples that result from its chunk of plan
+ * execution. This change may make the parallel plan cheap among all
+ * other plans, and influence the planner to consider this parallel
+ * plan.
+ */
+ if (!(IsA(path->subpath, ModifyTablePath) &&
+ castNode(ModifyTablePath, path->subpath)->operation == CMD_INSERT &&
+ castNode(ModifyTablePath, path->subpath)->returningLists != NULL))
+ {
+ run_cost += parallel_tuple_cost * path->path.rows;
+ }
Isn't the last condition in above check "castNode(ModifyTablePath,
path->subpath)->returningLists != NULL" should be
"castNode(ModifyTablePath, path->subpath)->returningLists == NULL"
instead? Because otherwise when there is returning list it won't count
the cost for passing tuples via Gather node. This might be reason of
what Thomas has seen in his recent email [2]/messages/by-id/CA+hUKGLZB=1Q+AQQEEmffr3bUMAh+JD+J+7axv+K10Kea0U9TQ@mail.gmail.com.
10. Don't we need a change similar to cost_gather in
cost_gather_merge? It seems you have made only partial changes for
GatherMerge node.
[1]: /messages/by-id/CAA4eK1KyftVDgovvRQmdV1b=nN0R-KqdWZqiu7jZ1GYQ7SO9OA@mail.gmail.com
[2]: /messages/by-id/CA+hUKGLZB=1Q+AQQEEmffr3bUMAh+JD+J+7axv+K10Kea0U9TQ@mail.gmail.com
--
With Regards,
Amit Kapila.
On Sat, Oct 10, 2020 at 5:25 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
8. You have made changes related to trigger execution for Gather node,
don't we need similar changes for GatherMerge node?
..
10. Don't we need a change similar to cost_gather in
cost_gather_merge? It seems you have made only partial changes for
GatherMerge node.
Please ignore these two comments as we can't push Insert to workers
when GatherMerge is involved as a leader backend does the final phase
(merge the results by workers). So, we can only push the Select part
of the statement.
--
With Regards,
Amit Kapila.
On Sun, Oct 11, 2020 at 12:55 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
+ /* + * For Parallel INSERT, provided no tuples are returned from workers + * to gather/leader node, don't add a cost-per-row, as each worker + * parallelly inserts the tuples that result from its chunk of plan + * execution. This change may make the parallel plan cheap among all + * other plans, and influence the planner to consider this parallel + * plan. + */ + if (!(IsA(path->subpath, ModifyTablePath) && + castNode(ModifyTablePath, path->subpath)->operation == CMD_INSERT && + castNode(ModifyTablePath, path->subpath)->returningLists != NULL)) + { + run_cost += parallel_tuple_cost * path->path.rows; + }Isn't the last condition in above check "castNode(ModifyTablePath,
path->subpath)->returningLists != NULL" should be
"castNode(ModifyTablePath, path->subpath)->returningLists == NULL"
instead? Because otherwise when there is returning list it won't count
the cost for passing tuples via Gather node. This might be reason of
what Thomas has seen in his recent email [2].
Yeah, I think this is trying to fix the problem too late. Instead, we
should fix the incorrect row estimates so we don't have to fudge it
later like that. For example, this should be estimating rows=0:
postgres=# explain analyze insert into s select * from t t1 join t t2 using (i);
...
Insert on s (cost=30839.08..70744.45 rows=1000226 width=4) (actual
time=2940.560..2940.562 rows=0 loops=1)
I think that should be done with something like this:
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3583,16 +3583,11 @@ create_modifytable_path(PlannerInfo *root,
RelOptInfo *rel,
if (lc == list_head(subpaths)) /* first node? */
pathnode->path.startup_cost = subpath->startup_cost;
pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
+ if (returningLists != NIL)
+ pathnode->path.rows += subpath->rows;
total_size += subpath->pathtarget->width * subpath->rows;
}
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
if (pathnode->path.rows > 0)
total_size /= pathnode->path.rows;
pathnode->path.pathtarget->width = rint(total_size);
On Sat, Oct 10, 2020 at 3:32 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
OK, for the minimal patch, just allowing INSERT with parallel SELECT,
you're right, neither of those additional "commandType == CMD_SELECT"
checks are needed, so I'll remove them.various
Okay, that makes sense.
For the minimal patch (just allowing INSERT with parallel SELECT),
there are issues with parallel-mode and various parallel-mode-related
checks in the code.
Initially, I thought it was only a couple of XID-related checks (which
could perhaps just be tweaked to check for IsParallelWorker() instead,
as you suggested), but I now realise that there are a lot more cases.
This stems from the fact that just having a parallel SELECT (as part
of non-parallel INSERT) causes parallel-mode to be set for the WHOLE
plan. I'm not sure why parallel-mode is set globally like this, for
the whole plan. Couldn't it just be set for the scope of
Gather/GatherMerge? Otherwise, errors from these checks seem to be
misleading when outside the scope of Gather/GatherMerge, as
technically they are not occurring within the scope of parallel-leader
and parallel-worker(s). The global parallel-mode wouldn't have been an
issue before, because up to now INSERT has never had underlying
parallel operations.
For example, when running the tests under
"force_parallel_mode=regress", the test failures show that there are a
lot more cases affected:
"cannot assign TransactionIds during a parallel operation"
"cannot assign XIDs during a parallel operation"
"cannot start commands during a parallel operation"
"cannot modify commandid in active snapshot during a parallel operation"
"cannot execute nextval() during a parallel operation"
"cannot execute INSERT during a parallel operation"
"cannot execute ANALYZE during a parallel operation
"cannot update tuples during a parallel operation"
(and there are more not currently detected by the tests, found by
searching the code).
As an example, with the minimal patch applied, if you had a trigger on
INSERT that, say, attempted a table creation or UPDATE/DELETE, and you
ran an "INSERT INTO ... SELECT...", it would treat the trigger
operations as being attempted in parallel-mode, and so an error would
result.
Let me know your thoughts on how to deal with these issues.
Can you see a problem with only having parallel-mode set for scope of
Gather/GatherMerge, or do you have some other idea?
Regards,
Greg Nancarrow
Fujitsu Australia
On Sun, Oct 11, 2020 at 1:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Oct 10, 2020 at 5:25 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
8. You have made changes related to trigger execution for Gather node,
don't we need similar changes for GatherMerge node?..
10. Don't we need a change similar to cost_gather in
cost_gather_merge? It seems you have made only partial changes for
GatherMerge node.Please ignore these two comments as we can't push Insert to workers
when GatherMerge is involved as a leader backend does the final phase
(merge the results by workers). So, we can only push the Select part
of the statement.
Precisely, that's why I didn't make those changes for GatherMerge.
Regards,
Greg Nancarrow
Fujitsu Australia
On Sun, Oct 11, 2020 at 1:39 PM Thomas Munro <thomas.munro@gmail.com> wrote:
Yeah, I think this is trying to fix the problem too late. Instead, we
should fix the incorrect row estimates so we don't have to fudge it
later like that. For example, this should be estimating rows=0:postgres=# explain analyze insert into s select * from t t1 join t t2 using (i);
...
Insert on s (cost=30839.08..70744.45 rows=1000226 width=4) (actual
time=2940.560..2940.562 rows=0 loops=1)I think that should be done with something like this:
--- a/src/backend/optimizer/util/pathnode.c +++ b/src/backend/optimizer/util/pathnode.c @@ -3583,16 +3583,11 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel, if (lc == list_head(subpaths)) /* first node? */ pathnode->path.startup_cost = subpath->startup_cost; pathnode->path.total_cost += subpath->total_cost; - pathnode->path.rows += subpath->rows; + if (returningLists != NIL) + pathnode->path.rows += subpath->rows; total_size += subpath->pathtarget->width * subpath->rows; }- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
if (pathnode->path.rows > 0)
total_size /= pathnode->path.rows;
pathnode->path.pathtarget->width = rint(total_size);
Agree, thanks (bug in existing Postgres code, right?)
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Oct 12, 2020 at 3:42 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sun, Oct 11, 2020 at 1:39 PM Thomas Munro <thomas.munro@gmail.com> wrote:
pathnode->path.total_cost += subpath->total_cost; - pathnode->path.rows += subpath->rows; + if (returningLists != NIL) + pathnode->path.rows += subpath->rows; total_size += subpath->pathtarget->width * subpath->rows; }
Erm, except the condition should of course cover total_size too.
Agree, thanks (bug in existing Postgres code, right?)
Yeah, I think we should go ahead and fix that up front. Here's a
version with a commit message.
Attachments:
0001-Fix-row-estimate-for-ModifyTable-paths.patchtext/x-patch; charset=US-ASCII; name=0001-Fix-row-estimate-for-ModifyTable-paths.patchDownload
From d92c45e6c1fa5ed118195de149d8fb833ea008ef Mon Sep 17 00:00:00 2001
From: Thomas Munro <thomas.munro@gmail.com>
Date: Mon, 12 Oct 2020 15:49:27 +1300
Subject: [PATCH] Fix row estimate for ModifyTable paths.
Historically, we estimated that a ModifyTable node would emit the same
number of rows as it writes. That's only true if there is a RETURNING
clause, otherwise the real number is zero. It didn't matter much
before, but proposed patches to allow for parallel writes revealed that
such queries would look unduly expensive if we continued to charge for a
phantom RETURNING clause.
Correct the estimates in master only. There doesn't seem to be much
point in back-patching the change for now.
Reviewed-by: Greg Nancarrow <gregn4422@gmail.com>
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV%3DqpFJrR3AcrTS3g%40mail.gmail.com
---
src/backend/optimizer/util/pathnode.c | 13 +++++--------
1 file changed, 5 insertions(+), 8 deletions(-)
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index c1fc866cbf..d718a4adba 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3583,16 +3583,13 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
if (lc == list_head(subpaths)) /* first node? */
pathnode->path.startup_cost = subpath->startup_cost;
pathnode->path.total_cost += subpath->total_cost;
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ if (returningLists != NIL)
+ {
+ pathnode->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
}
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should report zero if no RETURNING, else an average
- * of the RETURNING tlist widths. But it's what happened historically,
- * and improving it is a task for another day.
- */
if (pathnode->path.rows > 0)
total_size /= pathnode->path.rows;
pathnode->path.pathtarget->width = rint(total_size);
--
2.20.1
On Mon, Oct 12, 2020 at 6:51 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Oct 10, 2020 at 3:32 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
OK, for the minimal patch, just allowing INSERT with parallel SELECT,
you're right, neither of those additional "commandType == CMD_SELECT"
checks are needed, so I'll remove them.various
Okay, that makes sense.For the minimal patch (just allowing INSERT with parallel SELECT),
there are issues with parallel-mode and various parallel-mode-related
checks in the code.
Initially, I thought it was only a couple of XID-related checks (which
could perhaps just be tweaked to check for IsParallelWorker() instead,
as you suggested), but I now realise that there are a lot more cases.
This stems from the fact that just having a parallel SELECT (as part
of non-parallel INSERT) causes parallel-mode to be set for the WHOLE
plan. I'm not sure why parallel-mode is set globally like this, for
the whole plan. Couldn't it just be set for the scope of
Gather/GatherMerge? Otherwise, errors from these checks seem to be
misleading when outside the scope of Gather/GatherMerge, as
technically they are not occurring within the scope of parallel-leader
and parallel-worker(s). The global parallel-mode wouldn't have been an
issue before, because up to now INSERT has never had underlying
parallel operations.
That is right but there is another operation which works like that.
For ex. a statement like "create table test_new As select * from
test_parallel where c1 < 1000;" will use parallel select but the write
operation will be performed in a leader. I agree that the code flow of
Insert is different so we will have a different set of challenges in
that case but to make it work there shouldn't be any fundamental
problem.
For example, when running the tests under
"force_parallel_mode=regress", the test failures show that there are a
lot more cases affected:"cannot assign TransactionIds during a parallel operation"
"cannot assign XIDs during a parallel operation"
"cannot start commands during a parallel operation"
"cannot modify commandid in active snapshot during a parallel operation"
"cannot execute nextval() during a parallel operation"
"cannot execute INSERT during a parallel operation"
"cannot execute ANALYZE during a parallel operation
"cannot update tuples during a parallel operation"(and there are more not currently detected by the tests, found by
searching the code).
Did you get these after applying your patch? If so, can you share the
version which you are using, or if you have already posted the same
then point me to the same?
As an example, with the minimal patch applied, if you had a trigger on
INSERT that, say, attempted a table creation or UPDATE/DELETE, and you
ran an "INSERT INTO ... SELECT...", it would treat the trigger
operations as being attempted in parallel-mode, and so an error would
result.
Oh, I guess this happens because you need to execute Insert in
parallel-mode even though Insert is happening in the leader, right?
And probably we are not facing this with "Create Table As .." because
there is no trigger execution involved there.
Let me know your thoughts on how to deal with these issues.
Can you see a problem with only having parallel-mode set for scope of
Gather/GatherMerge, or do you have some other idea?
I have not thought about this yet but I don't understand your
proposal. How will you set it only for the scope of Gather (Merge)?
The execution of the Gather node will be interleaved with the Insert
node, basically, you fetch a tuple from Gather, and then you need to
Insert it. Can you be a bit more specific on what you have in mind for
this?
--
With Regards,
Amit Kapila.
On Mon, Oct 12, 2020 at 2:11 PM Thomas Munro <thomas.munro@gmail.com> wrote:
On Mon, Oct 12, 2020 at 3:42 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sun, Oct 11, 2020 at 1:39 PM Thomas Munro <thomas.munro@gmail.com> wrote:
pathnode->path.total_cost += subpath->total_cost; - pathnode->path.rows += subpath->rows; + if (returningLists != NIL) + pathnode->path.rows += subpath->rows; total_size += subpath->pathtarget->width * subpath->rows; }Erm, except the condition should of course cover total_size too.
Agree, thanks (bug in existing Postgres code, right?)
Yeah, I think we should go ahead and fix that up front. Here's a
version with a commit message.
I've checked it and tested it, and it looks fine to me.
Also, it seems to align with the gripe in the old comment about width
("XXX this is totally wrong: we should report zero if no RETURNING
...").
I'm happy for you to commit it.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Oct 12, 2020 at 9:01 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Oct 12, 2020 at 6:51 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Let me know your thoughts on how to deal with these issues.
Can you see a problem with only having parallel-mode set for scope of
Gather/GatherMerge, or do you have some other idea?I have not thought about this yet but I don't understand your
proposal. How will you set it only for the scope of Gather (Merge)?
The execution of the Gather node will be interleaved with the Insert
node, basically, you fetch a tuple from Gather, and then you need to
Insert it. Can you be a bit more specific on what you have in mind for
this?
One more thing I would like to add here is that we can't exit
parallel-mode till the workers are running (performing the scan or
other operation it is assigned with) and shared memory is not
destroyed. Otherwise, the leader can perform un-safe things like
assigning new commandsids or probably workers can send some error for
which the leader should still be in parallel-mode. So, considering
this I think we need quite similar checks (similar to parallel
inserts) to make even the Select part parallel for Inserts. If we do
that then you won't face many of the problems you mentioned above like
executing triggers that contain parallel-unsafe stuff. I feel still it
will be beneficial to do this as it will cover cases like Insert with
GatherMerge underneath it which would otherwise not possible.
--
With Regards,
Amit Kapila.
On Mon, Oct 12, 2020 at 5:36 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I have not thought about this yet but I don't understand your
proposal. How will you set it only for the scope of Gather (Merge)?
The execution of the Gather node will be interleaved with the Insert
node, basically, you fetch a tuple from Gather, and then you need to
Insert it. Can you be a bit more specific on what you have in mind for
this?One more thing I would like to add here is that we can't exit
parallel-mode till the workers are running (performing the scan or
other operation it is assigned with) and shared memory is not
destroyed. Otherwise, the leader can perform un-safe things like
assigning new commandsids or probably workers can send some error for
which the leader should still be in parallel-mode. So, considering
this I think we need quite similar checks (similar to parallel
inserts) to make even the Select part parallel for Inserts. If we do
that then you won't face many of the problems you mentioned above like
executing triggers that contain parallel-unsafe stuff. I feel still it
will be beneficial to do this as it will cover cases like Insert with
GatherMerge underneath it which would otherwise not possible.
Yes, I see what you mean, exiting parallel-mode can't be done safely
where I had hoped it could, so looks like, even for making just the
Select part of Insert parallel, I need to add checks (along the same
lines as for Parallel Insert) to avoid the parallel Select in certain
potentially-unsafe cases.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Oct 12, 2020 at 12:38 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Oct 12, 2020 at 5:36 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I have not thought about this yet but I don't understand your
proposal. How will you set it only for the scope of Gather (Merge)?
The execution of the Gather node will be interleaved with the Insert
node, basically, you fetch a tuple from Gather, and then you need to
Insert it. Can you be a bit more specific on what you have in mind for
this?One more thing I would like to add here is that we can't exit
parallel-mode till the workers are running (performing the scan or
other operation it is assigned with) and shared memory is not
destroyed. Otherwise, the leader can perform un-safe things like
assigning new commandsids or probably workers can send some error for
which the leader should still be in parallel-mode. So, considering
this I think we need quite similar checks (similar to parallel
inserts) to make even the Select part parallel for Inserts. If we do
that then you won't face many of the problems you mentioned above like
executing triggers that contain parallel-unsafe stuff. I feel still it
will be beneficial to do this as it will cover cases like Insert with
GatherMerge underneath it which would otherwise not possible.Yes, I see what you mean, exiting parallel-mode can't be done safely
where I had hoped it could, so looks like, even for making just the
Select part of Insert parallel, I need to add checks (along the same
lines as for Parallel Insert) to avoid the parallel Select in certain
potentially-unsafe cases.
Right, after we take care of that, we can think of assigning xid or
things like that before we enter parallel mode. Say we have a function
like PrepareParallelMode (or PrepareEnterParallelMode) or something
like that where we can check whether we need to perform
parallel-safe-write operation (as of now Insert) and then do the
required preparation like assign xid, etc. I think this might not be
idle because it is possible that we don't fetch even a single row (say
due to filter condition) which needs to be inserted and then we will
waste xid but such cases might not occur often enough to worry.
--
With Regards,
Amit Kapila.
On Mon, Oct 12, 2020 at 6:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Oct 12, 2020 at 2:11 PM Thomas Munro <thomas.munro@gmail.com> wrote:
Yeah, I think we should go ahead and fix that up front. Here's a
version with a commit message.I've checked it and tested it, and it looks fine to me.
Also, it seems to align with the gripe in the old comment about width
("XXX this is totally wrong: we should report zero if no RETURNING
...").
I'm happy for you to commit it.
Pushed, though I left most of that comment there because the width
estimate still needs work when you do use RETURNING. At least we now
have rows=0 for queries without RETURNING, which was a bigger problem
for your patch.
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
It might be a good idea to first just get this patch committed, if
possible. So, I have reviewed the latest version of this patch:0001-InsertParallelSelect
I've attached an updated InsertParallelSelect patch (v2) - allowing
underlying parallel SELECT for "INSERT INTO ... SELECT ...".
I think I've addressed most of the issues identified in the previous
version of the patch.
I'm still seeing a couple of errors in the tests when
"force_parallel_mode=regress" is in effect, and those need to be
addressed (extra checks required to avoid parallel SELECT in certain
cases).
Also, I'm seeing a partition-related error when running
installcheck-world - I'm investigating that.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v2-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v2-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From b8990055f96a052bc0d6609df59d072e02ec0db0 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 15 Oct 2020 13:59:25 +1100
Subject: [PATCH v2] Enable parallel SELECT for "INSERT INTO ... SELECT ...".
Enable "INSERT INTO ... SELECT ..." to utilize a parallel SELECT, where it is safe to do so.
Parallel SELECT can't be utilized in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Row-level trigger or transition-table trigger
- Parallel-unsafe trigger
- Foreign key trigger (RI_TRIGGER_FK)
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/varsup.c | 5 +-
src/backend/access/transam/xact.c | 4 +-
src/backend/optimizer/plan/planner.c | 295 ++++++++++++++++++++++++++++++++++-
3 files changed, 294 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index a4944fa..925c875 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -15,6 +15,7 @@
#include "access/clog.h"
#include "access/commit_ts.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/xact.h"
@@ -56,8 +57,8 @@ GetNewTransactionId(bool isSubXact)
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs after that point.
*/
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+ if (IsParallelWorker())
+ elog(ERROR, "cannot assign TransactionIds in a parallel worker");
/*
* During bootstrap initialization, we return the special bootstrap
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..ef423fb 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -580,8 +580,8 @@ AssignTransactionId(TransactionState s)
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs at this point.
*/
- if (IsInParallelMode() || IsParallelWorker())
- elog(ERROR, "cannot assign XIDs during a parallel operation");
+ if (IsParallelWorker())
+ elog(ERROR, "cannot assign XIDs in a parallel worker");
/*
* Ensure parent(s) have XIDs, so that a child always has an XID later
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f331f82..3bd2e22 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -24,10 +24,12 @@
#include "access/sysattr.h"
#include "access/table.h"
#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_inherits.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
#include "foreign/fdwapi.h"
@@ -58,6 +60,7 @@
#include "parser/parse_agg.h"
#include "parser/parsetree.h"
#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
#include "storage/dsm_impl.h"
#include "utils/lsyscache.h"
@@ -248,7 +251,11 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *targetList,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
+static bool IsTriggerDataParallelModeSafe(TriggerDesc *trigdesc);
+static bool IsRelParallelModeSafeForModify(Oid relid);
+static bool AreIndexExprsParallelModeSafe(Relation rel);
+static bool IsParallelModeSafeForModify(Query *parse);
+static bool IsModifySupportedInParallelMode(CmdType commandType);
/*****************************************************************************
*
@@ -319,11 +326,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
+ * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and CREATE
* MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
* backend writes into a completely new table. In the future, we can
* extend it to allow workers to write into the table. However, to allow
@@ -337,7 +344,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -345,6 +353,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->parallelModeOK = IsParallelModeSafeForModify(parse);
+ if (!glob->parallelModeOK)
+ glob->maxParallelHazard = PROPARALLEL_UNSAFE;
+ }
}
else
{
@@ -7355,6 +7371,273 @@ can_partial_agg(PlannerInfo *root, const AggClauseCosts *agg_costs)
}
/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * conditions.
+ */
+static pg_attribute_always_inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
+
+/*
+ * IsTriggerDataParallelModeSafe
+ *
+ * Checks if the specified trigger data is parallel-mode safe.
+ * Returns false if any one of the triggers are not safe for parallel
+ * operation.
+ */
+static bool
+IsTriggerDataParallelModeSafe(TriggerDesc *trigdesc)
+{
+ int i;
+
+ /*
+ * Can't support execution of row-level or transition-table triggers
+ * during parallel-mode, since such triggers may query the table
+ * into which the data is being inserted, and the content returned
+ * would vary unpredictably according to the order of retrieval by
+ * the workers and the rows already inserted.
+ */
+ if (trigdesc != NULL &&
+ (trigdesc->trig_insert_instead_row ||
+ trigdesc->trig_insert_before_row ||
+ trigdesc->trig_insert_after_row ||
+ trigdesc->trig_insert_new_table))
+ {
+ return false;
+ }
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (func_parallel(trigger->tgfoid) != PROPARALLEL_SAFE)
+ return false;
+
+ /* If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported during
+ * parallel-mode.
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ return false;
+ }
+
+ return true;
+}
+
+/*
+ * AreIndexExprsParallelModeSafe
+ *
+ * Checks if index expressions for a specified relation are parallel-mode safe.
+ * Returns false if any index expressions exist which are not safe for parallel
+ * operation.
+ */
+static bool
+AreIndexExprsParallelModeSafe(Relation rel)
+{
+ List *indexOidList;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ indexOidList = RelationGetIndexList(rel);
+ foreach(lc, indexOidList)
+ {
+ Oid indexOid = lfirst_oid(lc);
+ Relation indexRel;
+ IndexInfo *indexInfo;
+
+ indexRel = index_open(indexOid, lockmode);
+
+ indexInfo = BuildIndexInfo(indexRel);
+
+ if (indexInfo->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *indexExprItem = list_head(indexInfo->ii_Expressions);
+
+ for (i = 0; i < indexInfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexInfo->ii_IndexAttrNumbers[i];
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *indexExpr;
+
+ if (indexExprItem == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ indexExpr = (Node *)lfirst(indexExprItem);
+ indexExpr = (Node *)expression_planner((Expr *)indexExpr);
+
+ if (max_parallel_hazard((Query *)indexExpr) != PROPARALLEL_SAFE)
+ {
+ index_close(indexRel, lockmode);
+ return false;
+ }
+
+ indexExprItem = lnext(indexInfo->ii_Expressions, indexExprItem);
+ }
+ }
+ }
+ index_close(indexRel, lockmode);
+ }
+
+ return true;
+}
+
+/*
+ * IsRelParallelModeSafeForModify
+ *
+ * Determines whether a specified relation is safe for modification in
+ * parallel-mode.
+ */
+static bool
+IsRelParallelModeSafeForModify(Oid relid)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ rel = table_open(relid, lockmode);
+
+ /*
+ * We can't support table modification in parallel-mode if it's a
+ * foreign table/partition (no FDW API for supporting parallel access)
+ * or a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pd = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pd->nparts; i++)
+ {
+ if (!IsRelParallelModeSafeForModify(pd->oids[i]))
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (!AreIndexExprsParallelModeSafe(rel))
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ !IsTriggerDataParallelModeSafe(rel->trigdesc))
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+
+ /*
+ * Check if there are any column default expressions which are not
+ * parallel-mode safe.
+ */
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Expr *defexpr;
+
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ if (att->atthasdef)
+ {
+ defexpr = (Expr *)build_column_default(rel, attnum + 1);
+
+ /* Run the expression through planner */
+ defexpr = expression_planner(defexpr);
+
+ if (max_parallel_hazard((Query *)defexpr) != PROPARALLEL_SAFE)
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *checkExpr = stringToNode(check->ccbin);
+ if (max_parallel_hazard((Query *)checkExpr) != PROPARALLEL_SAFE)
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return true;
+}
+
+/*
+ * IsParallelModeSafeForModify
+ *
+ * Determines whether the specified table-modification statement is
+ * parallel-mode safe, based on the statement attributes and target table.
+ */
+static bool
+IsParallelModeSafeForModify(Query *parse)
+{
+ RangeTblEntry *rte;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (IsRelParallelModeSafeForModify(rte->relid));
+}
+
+/*
* apply_scanjoin_target_to_paths
*
* Adjust the final scan/join relation, and recursively all of its children,
--
1.8.3.1
On Thu, Oct 15, 2020 at 9:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 9, 2020 at 8:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
It might be a good idea to first just get this patch committed, if
possible. So, I have reviewed the latest version of this patch:0001-InsertParallelSelect
I've attached an updated InsertParallelSelect patch (v2) - allowing
underlying parallel SELECT for "INSERT INTO ... SELECT ...".
I think I've addressed most of the issues identified in the previous
version of the patch.
I'm still seeing a couple of errors in the tests when
"force_parallel_mode=regress" is in effect, and those need to be
addressed (extra checks required to avoid parallel SELECT in certain
cases).
I am getting below error in force_parallel_mode:
@@ -1087,9 +1087,14 @@
ERROR: value for domain inotnull violates check constraint "inotnull_check"
create table dom_table (x inotnull);
insert into dom_table values ('1');
+ERROR: cannot start commands during a parallel operation
+CONTEXT: SQL function "sql_is_distinct_from"
It happened with below test:
create function sql_is_distinct_from(anyelement, anyelement)
returns boolean language sql
as 'select $1 is distinct from $2 limit 1';
create domain inotnull int
check (sql_is_distinct_from(value, null));
create table dom_table (x inotnull);
insert into dom_table values ('1');
So it is clear that this is happening because we have allowed insert
that is parallel-unsafe. The attribute is of type domain which has a
parallel-unsafe constraint. As per your current code, we need to
detect it in IsRelParallelModeSafeForModify. The idea would be to
check the type of each attribute and if it is domain type then we need
to check if it has a constraint (See function ExecGrant_Type on how to
detect a domain type and then refer to functions
AlterTypeNamespaceInternal and AlterConstraintNamespaces to know how
to determine constraint for domain type). Once you can find a
constraint then you already have code in your patch to find if it
contains parallel-unsafe expression.
Also, I'm seeing a partition-related error when running
installcheck-world - I'm investigating that.
Okay.
Few more comments:
==================
1.
+ /*
+ * Can't support execution of row-level or transition-table triggers
+ * during parallel-mode, since such triggers may query the table
+ * into which the data is being inserted, and the content returned
+ * would vary unpredictably according to the order of retrieval by
+ * the workers and the rows already inserted.
+ */
+ if (trigdesc != NULL &&
+ (trigdesc->trig_insert_instead_row ||
+ trigdesc->trig_insert_before_row ||
+ trigdesc->trig_insert_after_row ||
+ trigdesc->trig_insert_new_table))
+ {
+ return false;
+ }
I don't think it is a good idea to prohibit all before/after/instead
row triggers because for the case you are referring to should mark
trigger functions as parallel-unsafe. We might want to have to Assert
somewhere to detect if there is illegal usage but I don't see the need
to directly prohibit them.
2.
@@ -56,8 +57,8 @@ GetNewTransactionId(bool isSubXact)
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs after that point.
*/
- if (IsInParallelMode())
- elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+ if (IsParallelWorker())
+ elog(ERROR, "cannot assign TransactionIds in a parallel worker");
/*
* During bootstrap initialization, we return the special bootstrap
diff --git a/src/backend/access/transam/xact.c
b/src/backend/access/transam/xact.c
index af6afce..ef423fb 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -580,8 +580,8 @@ AssignTransactionId(TransactionState s)
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs at this point.
*/
- if (IsInParallelMode() || IsParallelWorker())
- elog(ERROR, "cannot assign XIDs during a parallel operation");
+ if (IsParallelWorker())
+ elog(ERROR, "cannot assign XIDs in a parallel worker");
I think we don't need these changes at least for the purpose of this
patch if you follow the suggestion related to having a new function
like PrepareParallelMode in the email above [1]/messages/by-id/CAA4eK1JogfXUa=3wMPO+K=UiOLgHgCO7-fj1wCHsSxdaXsfVbw@mail.gmail.com. One problem I see
with removing these checks is how do we ensure that leader won't
assign a new transactionid once we start executing a parallel node. It
can do via sub-transactions maybe that is already protected at some
previous point but I would like to see if we can retain these checks.
[1]: /messages/by-id/CAA4eK1JogfXUa=3wMPO+K=UiOLgHgCO7-fj1wCHsSxdaXsfVbw@mail.gmail.com
--
With Regards,
Amit Kapila.
On Thu, Oct 15, 2020 at 6:13 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Oct 15, 2020 at 9:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Also, I'm seeing a partition-related error when running
installcheck-world - I'm investigating that.Okay.
The attached patch fixes this partition case for me. Basically, we
need to check the parallel-safety of PartitionKey. I have only checked
for partsupfunc but I think we need to check the parallel-safety of
partexprs as well. Also, I noticed that you have allowed for
parallelism only when all expressions/functions involved with Insert
are parallel-safe, can't we allow parallel-restricted case because
anyway Inserts have to be performed by the leader for this patch.
--
With Regards,
Amit Kapila.
Attachments:
fix_paratition_failure_1.patchapplication/octet-stream; name=fix_paratition_failure_1.patchDownload
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index e2f92aae08..43dab5c083 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -64,6 +64,7 @@
#include "rewrite/rewriteManip.h"
#include "storage/dsm_impl.h"
#include "utils/lsyscache.h"
+#include "utils/partcache.h"
#include "utils/rel.h"
#include "utils/selfuncs.h"
#include "utils/syscache.h"
@@ -7527,7 +7528,17 @@ IsRelParallelModeSafeForModify(Oid relid)
if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
{
int i;
- PartitionDesc pd = RelationGetPartitionDesc(rel);
+ PartitionDesc pd;
+
+ PartitionKey key = RelationGetPartitionKey(rel);
+
+ if (func_parallel(key->partsupfunc->fn_oid) != PROPARALLEL_SAFE)
+ {
+ table_close(rel, lockmode);
+ return false;
+ }
+
+ pd = RelationGetPartitionDesc(rel);
for (i = 0; i < pd->nparts; i++)
{
if (!IsRelParallelModeSafeForModify(pd->oids[i]))
On Fri, Oct 16, 2020 at 3:43 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Oct 15, 2020 at 6:13 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Oct 15, 2020 at 9:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Also, I'm seeing a partition-related error when running
installcheck-world - I'm investigating that.Okay.
The attached patch fixes this partition case for me. Basically, we
need to check the parallel-safety of PartitionKey. I have only checked
for partsupfunc but I think we need to check the parallel-safety of
partexprs as well.
Thanks, I had already added the parallel-safety check for partexprs
when I saw this, so your patch hopefully completes all the
partition-related checks required.
Also, I noticed that you have allowed for
parallelism only when all expressions/functions involved with Insert
are parallel-safe, can't we allow parallel-restricted case because
anyway Inserts have to be performed by the leader for this patch.
Yes, I think you're right.
"A parallel restricted operation is one which cannot be performed in a
parallel worker, but which can be performed in the leader while
parallel query is in use."
I'll make the change and test that everything works OK.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Oct 16, 2020 at 2:16 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 16, 2020 at 3:43 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Also, I noticed that you have allowed for
parallelism only when all expressions/functions involved with Insert
are parallel-safe, can't we allow parallel-restricted case because
anyway Inserts have to be performed by the leader for this patch.Yes, I think you're right.
"A parallel restricted operation is one which cannot be performed in a
parallel worker, but which can be performed in the leader while
parallel query is in use."
I'll make the change and test that everything works OK.
Cool, let me try to explain my thoughts a bit more. The idea is first
(in standard_planner) we check if there is any 'parallel_unsafe'
function/expression (via max_parallel_hazard) in the query tree. If we
don't find anything 'parallel_unsafe' then we mark parallelModeOk. At
this stage, the patch is checking whether there is any
'parallel_unsafe' or 'parallel_restricted' expression/function in the
target relation and if there is none then we mark parallelModeOK as
true. So, if there is anything 'parallel_restricted' then we will mark
parallelModeOK as false which doesn't seem right to me.
Then later in the planner during set_rel_consider_parallel, we
determine if a particular relation can be scanned from within a
worker, then we consider that relation for parallelism. Here, we
determine if certain things are parallel-restricted then we don't
consider this for parallelism. Then we create partial paths for the
relations that are considered for parallelism. I think we don't need
to change anything for the current patch in these later stages because
we anyway are not considering Insert to be pushed into workers.
However, in the second patch where we are thinking to push Inserts in
workers, we might need to do something to filter parallel-restricted
cases during this stage of the planner.
--
With Regards,
Amit Kapila.
On Fri, Oct 16, 2020 at 9:26 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Cool, let me try to explain my thoughts a bit more. The idea is first
(in standard_planner) we check if there is any 'parallel_unsafe'
function/expression (via max_parallel_hazard) in the query tree. If we
don't find anything 'parallel_unsafe' then we mark parallelModeOk. At
this stage, the patch is checking whether there is any
'parallel_unsafe' or 'parallel_restricted' expression/function in the
target relation and if there is none then we mark parallelModeOK as
true. So, if there is anything 'parallel_restricted' then we will mark
parallelModeOK as false which doesn't seem right to me.Then later in the planner during set_rel_consider_parallel, we
determine if a particular relation can be scanned from within a
worker, then we consider that relation for parallelism. Here, we
determine if certain things are parallel-restricted then we don't
consider this for parallelism. Then we create partial paths for the
relations that are considered for parallelism. I think we don't need
to change anything for the current patch in these later stages because
we anyway are not considering Insert to be pushed into workers.
However, in the second patch where we are thinking to push Inserts in
workers, we might need to do something to filter parallel-restricted
cases during this stage of the planner.
Posting an update to the smaller patch (Parallel SELECT for INSERT
INTO...SELECT...).
Most of this patch feeds into the larger Parallel INSERT patch, for
which I'll also be posting an update soon.
Patch updates include:
- Removed explicit trigger-type checks (instead rely on declared
trigger parallel safety)
- Restored parallel-related XID checks that previous patch altered;
now assign XID prior to entering parallel-mode
- Now considers parallel-SELECT for parallel RESTRICTED cases (not
just parallel SAFE cases)
- Added parallel-safety checks for partition key expressions and
support functions
- Workaround added for test failure in "partition-concurrent-attach"
test; since ALTER TABLE operations may exclusively lock a relation
until end-of-transaction, now assume and return UNSAFE if can't
acquire a share-lock on the relation, rather than block until
potentially end of the other concurrent transaction in which the
exclusive lock is held.
Examples of when a relation is exclusively locked
(AccessExclusiveLock) until end-of-transaction include:
ALTER TABLE DROP COLUMN
ALTER TABLE ... RENAME
ALTER TABLE ... ATTACH PARTITION (locks default partition)
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v3-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v3-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 79e818f8041aeb57ae154e916b1844cb9fe427b9 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 22 Oct 2020 14:58:56 +1100
Subject: [PATCH v3] Enable parallel SELECT for "INSERT INTO ... SELECT ...",
where it is safe to do so.
Parallel SELECT can't be utilized in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Parallel-unsafe trigger
- Foreign key trigger (RI_TRIGGER_FK)
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features
- Partitioned table with parallel-unsafe partition key expressions or support functions
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 13 ++
src/backend/executor/execMain.c | 10 +
src/backend/optimizer/plan/planner.c | 23 +-
src/backend/optimizer/util/clauses.c | 416 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 473 insertions(+), 5 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..7c37be8 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1015,6 +1015,19 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelModeForModify
+ *
+ * Prepare for entering parallel mode by assigning a FullTransactionId, to be
+ * included in the transaction state that is serialized in the parallel DSM.
+ */
+void PrepareParallelModeForModify(CmdType commandType)
+{
+ Assert(!IsInParallelMode());
+
+ (void)GetCurrentFullTransactionId();
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index aea0479..1a2a675 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,17 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ /*
+ * Supported table-modification commands may require additional steps
+ * prior to entering parallel mode, such as assigning a FullTransactionId.
+ */
+ if (IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType))
+ {
+ PrepareParallelModeForModify(estate->es_plannedstmt->commandType);
+ }
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 986d7a5..7c8c3db 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -318,11 +318,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
+ * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and CREATE
* MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
* backend writes into a completely new table. In the future, we can
* extend it to allow workers to write into the table. However, to allow
@@ -336,7 +336,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -344,6 +345,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allowing an underlying parallel query to be used for a
+ * supported table-modification command.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = MaxParallelHazardForModify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index e7d8146..7a2b7dc 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -157,6 +169,13 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context);
+static char MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context);
+static char MaxRelParallelHazardForModify(Oid relid, CmdType commandType,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -1073,6 +1092,403 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * conditions.
+ */
+static pg_attribute_always_inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
+
+/*
+ * MaxTriggerDataParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported during
+ * parallel-mode.
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxIndexExprsParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *indexOidList;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ indexOidList = RelationGetIndexList(rel);
+ foreach(lc, indexOidList)
+ {
+ Oid indexOid = lfirst_oid(lc);
+ Relation indexRel;
+ IndexInfo *indexInfo;
+
+ if (ConditionalLockRelationOid(indexOid, lockmode))
+ {
+ indexRel = index_open(indexOid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ indexInfo = BuildIndexInfo(indexRel);
+
+ if (indexInfo->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *indexExprItem = list_head(indexInfo->ii_Expressions);
+
+ for (i = 0; i < indexInfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexInfo->ii_IndexAttrNumbers[i];
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *indexExpr;
+
+ if (indexExprItem == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ indexExpr = (Node *)lfirst(indexExprItem);
+ indexExpr = (Node *)expression_planner((Expr *)indexExpr);
+
+ if (max_parallel_hazard_walker(indexExpr, context) == PROPARALLEL_UNSAFE)
+ {
+ index_close(indexRel, lockmode);
+ return context->max_hazard;
+ }
+
+ indexExprItem = lnext(indexInfo->ii_Expressions, indexExprItem);
+ }
+ }
+ }
+ index_close(indexRel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxDomainParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation conRel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ conRel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(conRel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *checkExpr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ elog(ERROR, "null conbin for constraint %u", con->oid);
+ conbin = TextDatumGetCString(val);
+ checkExpr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(conRel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxRelParallelHazardForModify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+MaxRelParallelHazardForModify(Oid relid,
+ CmdType commandType,
+ max_parallel_hazard_context *context)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ /*
+ * It's possible that this relation is locked for exclusive access
+ * in another concurrent transaction (e.g. as a result of a
+ * ALTER TABLE ... operation) until that transaction completes.
+ * If a share-lock can't be acquired on it now, we have to assume this
+ * could be the worst-case, so to avoid blocking here until that
+ * transaction completes, conditionally try to acquire the lock and
+ * assume and return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a
+ * foreign table/partition (no FDW API for supporting parallel access)
+ * or a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *checkExpr = (Node *) lfirst(partexprs_item);
+ if (max_parallel_hazard_walker(checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (MaxRelParallelHazardForModify(pdesc->oids[i], commandType, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (MaxIndexExprsParallelHazardForModify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ MaxTriggerDataParallelHazardForModify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ if (commandType == CMD_INSERT || commandType == CMD_UPDATE)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that domain
+ * has any CHECK expressions that are not parallel-mode safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (MaxDomainParallelHazardForModify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not parallel-safe.
+ */
+ if ((commandType == CMD_INSERT || commandType == CMD_UPDATE) &&
+ tupdesc->constr != NULL &&
+ tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *checkExpr = stringToNode(check->ccbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxParallelHazardForModify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initialMaxParallelHazard == NULL ||
+ *initialMaxParallelHazard == PROPARALLEL_SAFE ||
+ *initialMaxParallelHazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initialMaxParallelHazard == NULL ?
+ PROPARALLEL_SAFE : *initialMaxParallelHazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (MaxRelParallelHazardForModify(rte->relid, parse->commandType, &context));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..a926fff 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModeForModify(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 7ef8cce..12662eb 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -55,5 +55,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard);
#endif /* CLAUSES_H */
--
1.8.3.1
On Fri, Oct 16, 2020 at 9:26 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Cool, let me try to explain my thoughts a bit more. The idea is first
(in standard_planner) we check if there is any 'parallel_unsafe'
function/expression (via max_parallel_hazard) in the query tree. If we
don't find anything 'parallel_unsafe' then we mark parallelModeOk. At
this stage, the patch is checking whether there is any
'parallel_unsafe' or 'parallel_restricted' expression/function in the
target relation and if there is none then we mark parallelModeOK as
true. So, if there is anything 'parallel_restricted' then we will mark
parallelModeOK as false which doesn't seem right to me.Then later in the planner during set_rel_consider_parallel, we
determine if a particular relation can be scanned from within a
worker, then we consider that relation for parallelism. Here, we
determine if certain things are parallel-restricted then we don't
consider this for parallelism. Then we create partial paths for the
relations that are considered for parallelism. I think we don't need
to change anything for the current patch in these later stages because
we anyway are not considering Insert to be pushed into workers.
However, in the second patch where we are thinking to push Inserts in
workers, we might need to do something to filter parallel-restricted
cases during this stage of the planner.
Posting an updated Parallel INSERT patch which (mostly) addresses
previously-identified issues and suggestions.
More work needs to be done in order to support parallel UPDATE and
DELETE (even after application of Thomas Munro's combo-cid
parallel-support patch), but it is getting closer.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v5-0001-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v5-0001-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 9b201036cfb18e1b729755ddc20e55b8b0f2686f Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Tue, 27 Oct 2020 14:07:13 +1100
Subject: [PATCH v5] Enable parallel INSERT and/or SELECT for "INSERT
INTO...SELECT...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Parallel-unsafe trigger
- Foreign key trigger (RI_TRIGGER_FK)
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features
- Partitioned table with parallel-unsafe partition key expressions or support
functions
Where the above-mentioned target table features are parallel-restricted, rather
than parallel-unsafe, at least parallel SELECT may be utilized.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 2 +
src/backend/access/transam/xact.c | 44 +++-
src/backend/executor/execMain.c | 15 +-
src/backend/executor/execParallel.c | 59 ++++-
src/backend/executor/nodeGather.c | 63 ++++-
src/backend/executor/nodeGatherMerge.c | 11 +-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 ++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 144 +++++++++++-
src/backend/optimizer/plan/setrefs.c | 12 +-
src/backend/optimizer/util/clauses.c | 405 ++++++++++++++++++++++++++++++++
src/backend/optimizer/util/pathnode.c | 55 ++---
src/include/access/xact.h | 16 ++
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 4 +-
src/include/optimizer/clauses.h | 1 +
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
20 files changed, 857 insertions(+), 74 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..929e2a1 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,12 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+/*
if (IsParallelWorker())
ereport(ERROR,
(errcode(ERRCODE_INVALID_TRANSACTION_STATE),
errmsg("cannot insert tuples in a parallel worker")));
+*/
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..7a6426a 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -764,12 +778,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed
+ * if currentCommandIdUsed was already true at the start of the
+ * parallel operation (by way of SetCurrentCommandIdUsed()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision
+ * for communicating this back to the leader. Once currentCommandIdUsed
+ * is set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1015,6 +1033,22 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelModeForModify
+ *
+ * Prepare for entering parallel mode by assigning a FullTransactionId, to be
+ * included in the transaction state that is serialized in the parallel DSM.
+ */
+void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader)
+{
+ Assert(!IsInParallelMode());
+
+ if (isParallelModifyLeader)
+ (void)GetCurrentCommandId(true);
+
+ (void)GetCurrentFullTransactionId();
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..0969a66 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1526,7 +1527,19 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ /*
+ * Supported table-modification commands may require additional steps
+ * prior to entering parallel mode, such as assigning a FullTransactionId.
+ */
+ if (IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType))
+ {
+ PrepareParallelModeForModify(estate->es_plannedstmt->commandType, isParallelModifyLeader);
+ }
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..39f60af 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Allocate space for each worker's returned "# of tuples processed" count. */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1178,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1414,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1436,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of
+ * the parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1504,13 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /* Report the # of tuples processed during parallel INSERT execution. */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..dfe5442 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if there
+ * are BEFORE STATEMENT triggers, these must be fired by the leader,
+ * not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause -
+ * no readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -418,14 +446,31 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ /*
+ * If the parallel context has already been destroyed, this
+ * function must have been previously called, so just
+ * return.
+ */
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ bool isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /* For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not
+ * by the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeGatherMerge.c b/src/backend/executor/nodeGatherMerge.c
index 4712934..9f4a700 100644
--- a/src/backend/executor/nodeGatherMerge.c
+++ b/src/backend/executor/nodeGatherMerge.c
@@ -1,4 +1,4 @@
-/*-------------------------------------------------------------------------
+/*------------------------------------------------------------------------
*
* nodeGatherMerge.c
* Scan a plan in multiple workers, and do order-preserving merge.
@@ -210,6 +210,15 @@ ExecGatherMerge(PlanState *pstate)
{
ParallelContext *pcxt;
+ if (IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType))
+ {
+ /*
+ * Assign FullTransactionId, to be included in the
+ * transaction state that is serialized in the DSM.
+ */
+ GetCurrentFullTransactionId();
+ }
+
/* Initialize, or re-initialize, shared state needed by workers. */
if (!node->pei)
node->pei = ExecInitParallelPlan(node->ps.lefttree,
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 29e07b7..46402d7 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1833,6 +1834,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local
+ * plan execution (ModifyTable processing). These will be
+ * fired at end of Gather processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2158,7 +2192,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2235,7 +2273,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 733f7ea..7aa10b9 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a task
+ * for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 94280a7..5893051 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 986d7a5..b0e9c34 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -318,11 +318,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
+ * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and CREATE
* MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
* backend writes into a completely new table. In the future, we can
* extend it to allow workers to write into the table. However, to allow
@@ -336,7 +336,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -344,6 +345,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allowing an underlying parallel query to be used for a
+ * supported table-modification command.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = MaxParallelHazardForModify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
@@ -1794,7 +1807,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1842,6 +1856,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2378,13 +2393,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel INSERT/UPDATE/DELETE,
+ * it seems resonable to use the same number of workers as estimated
+ * for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2401,6 +2505,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7570,7 +7680,25 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * Don't allow a supported parallel table-modification command,
+ * because it's not safe. However, do allow any underlying query
+ * to be run by parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 8b43371..921c4bc 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = outerPlan(finalPlan)->targetlist;
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index e7d8146..fad55e4 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -157,6 +169,13 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context);
+static char MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context);
+static char MaxRelParallelHazardForModify(Oid relid, CmdType commandType,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -1073,6 +1092,392 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * MaxTriggerDataParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+ /*
+ * As we're looking for the max parallel hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxIndexExprsParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *indexOidList;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ indexOidList = RelationGetIndexList(rel);
+ foreach(lc, indexOidList)
+ {
+ Oid indexOid = lfirst_oid(lc);
+ Relation indexRel;
+ IndexInfo *indexInfo;
+
+ if (ConditionalLockRelationOid(indexOid, lockmode))
+ {
+ indexRel = index_open(indexOid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ indexInfo = BuildIndexInfo(indexRel);
+
+ if (indexInfo->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *indexExprItem = list_head(indexInfo->ii_Expressions);
+
+ for (i = 0; i < indexInfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexInfo->ii_IndexAttrNumbers[i];
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *indexExpr;
+
+ if (indexExprItem == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ indexExpr = (Node *)lfirst(indexExprItem);
+ indexExpr = (Node *)expression_planner((Expr *)indexExpr);
+
+ if (max_parallel_hazard_walker(indexExpr, context) == PROPARALLEL_UNSAFE)
+ {
+ index_close(indexRel, lockmode);
+ return context->max_hazard;
+ }
+
+ indexExprItem = lnext(indexInfo->ii_Expressions, indexExprItem);
+ }
+ }
+ }
+ index_close(indexRel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxDomainParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation conRel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ conRel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(conRel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *checkExpr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ elog(ERROR, "null conbin for constraint %u", con->oid);
+ conbin = TextDatumGetCString(val);
+ checkExpr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(conRel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxRelParallelHazardForModify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+MaxRelParallelHazardForModify(Oid relid,
+ CmdType commandType,
+ max_parallel_hazard_context *context)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ /*
+ * It's possible that this relation is locked for exclusive access
+ * in another concurrent transaction (e.g. as a result of a
+ * ALTER TABLE ... operation) until that transaction completes.
+ * If a share-lock can't be acquired on it now, we have to assume this
+ * could be the worst-case, so to avoid blocking here until that
+ * transaction completes, conditionally try to acquire the lock and
+ * assume and return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a
+ * foreign table/partition (no FDW API for supporting parallel access)
+ * or a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *checkExpr = (Node *) lfirst(partexprs_item);
+ if (max_parallel_hazard_walker(checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (MaxRelParallelHazardForModify(pdesc->oids[i], commandType, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (MaxIndexExprsParallelHazardForModify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ MaxTriggerDataParallelHazardForModify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ if (commandType == CMD_INSERT || commandType == CMD_UPDATE)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that domain
+ * has any CHECK expressions that are not parallel-mode safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (MaxDomainParallelHazardForModify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not parallel-safe.
+ */
+ if ((commandType == CMD_INSERT || commandType == CMD_UPDATE) &&
+ tupdesc->constr != NULL &&
+ tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *checkExpr = stringToNode(check->ccbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxParallelHazardForModify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initialMaxParallelHazard == NULL ||
+ *initialMaxParallelHazard == PROPARALLEL_SAFE ||
+ *initialMaxParallelHazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initialMaxParallelHazard == NULL ?
+ PROPARALLEL_SAFE : *initialMaxParallelHazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (MaxRelParallelHazardForModify(rte->relid, parse->commandType, &context));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5281a2f..30dc022 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,21 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach (lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3589,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..f43a844 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,5 +467,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..afb8a57 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6c0a7d6..c558eef 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1175,8 +1175,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
-
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
* an UPDATE of a partitioned table.
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 7ef8cce..12662eb 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -55,5 +55,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard);
#endif /* CLAUSES_H */
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
--
1.8.3.1
On Thu, Oct 22, 2020 at 9:47 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Oct 16, 2020 at 9:26 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Posting an update to the smaller patch (Parallel SELECT for INSERT
INTO...SELECT...).Most of this patch feeds into the larger Parallel INSERT patch, for
which I'll also be posting an update soon.Patch updates include:
- Removed explicit trigger-type checks (instead rely on declared
trigger parallel safety)
- Restored parallel-related XID checks that previous patch altered;
now assign XID prior to entering parallel-mode
- Now considers parallel-SELECT for parallel RESTRICTED cases (not
just parallel SAFE cases)
- Added parallel-safety checks for partition key expressions and
support functions
- Workaround added for test failure in "partition-concurrent-attach"
test;
IIUC, below is code for this workaround:
+MaxRelParallelHazardForModify(Oid relid,
+ CmdType commandType,
+ max_parallel_hazard_context *context)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ /*
+ * It's possible that this relation is locked for exclusive access
+ * in another concurrent transaction (e.g. as a result of a
+ * ALTER TABLE ... operation) until that transaction completes.
+ * If a share-lock can't be acquired on it now, we have to assume this
+ * could be the worst-case, so to avoid blocking here until that
+ * transaction completes, conditionally try to acquire the lock and
+ * assume and return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
Do we need this workaround if we lock just the parent table instead of
locking all the tables? Basically, can we safely identify the
parallel-safety of partitioned relation if we just have a lock on
parent relation? One more thing I have noticed is that for scan
relations (Select query), we do such checks much later based on
RelOptInfo (see set_rel_consider_parallel) which seems to have most of
the information required to perform parallel-safety checks but I guess
for ModifyTable (aka the Insert table) the equivalent doesn't seem
feasible but have you thought of doing at the later stage in planner?
Few other comments on latest patch:
===============================
1.
MaxRelParallelHazardForModify()
{
..
+ if (commandType == CMD_INSERT || commandType == CMD_UPDATE)
+ {
+ /*
..
Why to check CMD_UPDATE here?
2.
+void PrepareParallelModeForModify(CmdType commandType, bool
isParallelModifyLeader)
+{
+ Assert(!IsInParallelMode());
+
+ if (isParallelModifyLeader)
+ (void)GetCurrentCommandId(true);
+
+ (void)GetCurrentFullTransactionId();
Here, we should use GetCurrentTransactionId() similar to heap_insert
or other heap operations. I am not sure why you have used
GetCurrentFullTransactionId?
3. Can we have a test to show why we need to check all the partitions
for parallel-safety? I think it would be possible when there is a
trigger on only one of the partitions and that trigger has
corresponding parallel_unsafe function. But it is good to verify that
once.
4. Have you checked the overhead of this on the planner for different
kinds of statements like inserts into tables having 100 or 500
partitions? Similarly, it is good to check the overhead of domain
related checks added in the patch.
5. Can we have a separate patch for parallel-selects for Insert? It
will make review easier.
--
With Regards,
Amit Kapila.
On Tue, Oct 27, 2020 at 8:56 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
IIUC, below is code for this workaround:
+MaxRelParallelHazardForModify(Oid relid, + CmdType commandType, + max_parallel_hazard_context *context) +{ + Relation rel; + TupleDesc tupdesc; + int attnum; + + LOCKMODE lockmode = AccessShareLock; + + /* + * It's possible that this relation is locked for exclusive access + * in another concurrent transaction (e.g. as a result of a + * ALTER TABLE ... operation) until that transaction completes. + * If a share-lock can't be acquired on it now, we have to assume this + * could be the worst-case, so to avoid blocking here until that + * transaction completes, conditionally try to acquire the lock and + * assume and return UNSAFE on failure. + */ + if (ConditionalLockRelationOid(relid, lockmode)) + { + rel = table_open(relid, NoLock); + } + else + { + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + }Do we need this workaround if we lock just the parent table instead of
locking all the tables? Basically, can we safely identify the
parallel-safety of partitioned relation if we just have a lock on
parent relation?
I believe the workaround is still needed in this case, because the
workaround was added because of a test in which the parent table was
exclusively locked in another concurrent transaction (as a result of
ALTER TABLE ... ATTACH PARTITION ...) so we could not even get a
ShareLock on the parent table without hanging (and then ending up
failing the test because of it).
So at the moment the workaround is needed, even if just trying to lock
the parent table.
I'll do some more testing to determine the secondary issue of whether
locks on the partition tables are needed, but at the moment I believe
they are.
One more thing I have noticed is that for scan
relations (Select query), we do such checks much later based on
RelOptInfo (see set_rel_consider_parallel) which seems to have most of
the information required to perform parallel-safety checks but I guess
for ModifyTable (aka the Insert table) the equivalent doesn't seem
feasible but have you thought of doing at the later stage in planner?
Yes, and in fact I tried putting the checks in a later stage of the
planner, and it's almost successful, except it then makes setting
"parallelModeNeeded" very tricky indeed, because that is expected to
be set based on whether the SQL is safe to run in parallel mode
(paralleModeOK == true) and whether force_parallel_mode is not off.
With parallel safety checks delayed to a later stage in the planner,
it's then not known whether there are certain types of parallel-unsafe
INSERTs (such as INSERT INTO ... VALUES ... ON CONFLICT DO UPDATE
...), because processing for those doesn't reach those later stages of
the planner where parallelism is being considered. So then to avoid
errors from when parallel-mode is forced on and such unsafe INSERTs
are run, the only real choice is to only allow parallelModeNeeded to
be true for SELECT only (not INSERT), and this is kind of cheating and
also not picking up cases where parallel-safe INSERT is run but
invokes parallel-mode-unsafe features.
My conclusion, at least for the moment, is to leave the check where it is.
Few other comments on latest patch: =============================== 1. MaxRelParallelHazardForModify() { .. + if (commandType == CMD_INSERT || commandType == CMD_UPDATE) + { + /* ..Why to check CMD_UPDATE here?
That was a bit of forward-thinking, for when/if UPDATE/DELETE is
supported in parallel-mode.
Column default expressions and check-constraints are only applicable
to INSERT and UPDATE.
Note however that currently this function can only ever be called with
commandType == CMD_INSERT.
2. +void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader) +{ + Assert(!IsInParallelMode()); + + if (isParallelModifyLeader) + (void)GetCurrentCommandId(true); + + (void)GetCurrentFullTransactionId();Here, we should use GetCurrentTransactionId() similar to heap_insert
or other heap operations. I am not sure why you have used
GetCurrentFullTransactionId?
GetCurrentTransactionId() and GetCurrentFullTransactionId() actually
have the same functionality, just a different return value (which is
not being used here).
But anyway I've changed it to use GetCurrentTransactionId().
3. Can we have a test to show why we need to check all the partitions
for parallel-safety? I think it would be possible when there is a
trigger on only one of the partitions and that trigger has
corresponding parallel_unsafe function. But it is good to verify that
once.
I can't imagine how you could check parallel-safety properly without
checking all of the partitions.
We don't know which partition that data will get inserted into until
runtime (e.g. range/list partitioning).
Each partition can have its own column default expressions,
check-constraints, triggers etc. (which may or may not be
parallel-safe) and a partition may itself be a partitioned table.
4. Have you checked the overhead of this on the planner for different
kinds of statements like inserts into tables having 100 or 500
partitions? Similarly, it is good to check the overhead of domain
related checks added in the patch.
Checking that now and will post results soon.
5. Can we have a separate patch for parallel-selects for Insert? It
will make review easier.
See attached patches.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v6-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v6-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 72f486ab6bee87d4579a13e65c46d5e98c54a3eb Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 30 Oct 2020 00:23:37 +1100
Subject: [PATCH v6 1/2] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Parallel-unsafe trigger
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features
- Partitioned table with parallel-unsafe partition key expressions or support
functions
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 13 ++
src/backend/executor/execMain.c | 10 +
src/backend/optimizer/plan/planner.c | 23 +-
src/backend/optimizer/util/clauses.c | 405 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 462 insertions(+), 5 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index af6afce..6130df1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1015,6 +1015,19 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelModeForModify
+ *
+ * Prepare for entering parallel mode by assigning a FullTransactionId, to be
+ * included in the transaction state that is serialized in the parallel DSM.
+ */
+void PrepareParallelModeForModify(CmdType commandType)
+{
+ Assert(!IsInParallelMode());
+
+ (void)GetCurrentTransactionId();
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..74a7960 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,17 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ /*
+ * Supported table-modification commands may require additional steps
+ * prior to entering parallel mode, such as assigning a FullTransactionId.
+ */
+ if (IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType))
+ {
+ PrepareParallelModeForModify(estate->es_plannedstmt->commandType);
+ }
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 986d7a5..93c5b6f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -318,11 +318,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
+ * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and CREATE
* MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
* backend writes into a completely new table. In the future, we can
* extend it to allow workers to write into the table. However, to allow
@@ -336,7 +336,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -344,6 +345,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = MaxParallelHazardForModify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index e7d8146..fad55e4 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -157,6 +169,13 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context);
+static char MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context);
+static char MaxRelParallelHazardForModify(Oid relid, CmdType commandType,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -1073,6 +1092,392 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * MaxTriggerDataParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+MaxTriggerDataParallelHazardForModify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+ /*
+ * As we're looking for the max parallel hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxIndexExprsParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+MaxIndexExprsParallelHazardForModify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *indexOidList;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ indexOidList = RelationGetIndexList(rel);
+ foreach(lc, indexOidList)
+ {
+ Oid indexOid = lfirst_oid(lc);
+ Relation indexRel;
+ IndexInfo *indexInfo;
+
+ if (ConditionalLockRelationOid(indexOid, lockmode))
+ {
+ indexRel = index_open(indexOid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ indexInfo = BuildIndexInfo(indexRel);
+
+ if (indexInfo->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *indexExprItem = list_head(indexInfo->ii_Expressions);
+
+ for (i = 0; i < indexInfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexInfo->ii_IndexAttrNumbers[i];
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *indexExpr;
+
+ if (indexExprItem == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ indexExpr = (Node *)lfirst(indexExprItem);
+ indexExpr = (Node *)expression_planner((Expr *)indexExpr);
+
+ if (max_parallel_hazard_walker(indexExpr, context) == PROPARALLEL_UNSAFE)
+ {
+ index_close(indexRel, lockmode);
+ return context->max_hazard;
+ }
+
+ indexExprItem = lnext(indexInfo->ii_Expressions, indexExprItem);
+ }
+ }
+ }
+ index_close(indexRel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * MaxDomainParallelHazardForModify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+MaxDomainParallelHazardForModify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation conRel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ conRel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(conRel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *checkExpr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ elog(ERROR, "null conbin for constraint %u", con->oid);
+ conbin = TextDatumGetCString(val);
+ checkExpr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(conRel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxRelParallelHazardForModify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+MaxRelParallelHazardForModify(Oid relid,
+ CmdType commandType,
+ max_parallel_hazard_context *context)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ /*
+ * It's possible that this relation is locked for exclusive access
+ * in another concurrent transaction (e.g. as a result of a
+ * ALTER TABLE ... operation) until that transaction completes.
+ * If a share-lock can't be acquired on it now, we have to assume this
+ * could be the worst-case, so to avoid blocking here until that
+ * transaction completes, conditionally try to acquire the lock and
+ * assume and return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a
+ * foreign table/partition (no FDW API for supporting parallel access)
+ * or a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *checkExpr = (Node *) lfirst(partexprs_item);
+ if (max_parallel_hazard_walker(checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (MaxRelParallelHazardForModify(pdesc->oids[i], commandType, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (MaxIndexExprsParallelHazardForModify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ MaxTriggerDataParallelHazardForModify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ if (commandType == CMD_INSERT || commandType == CMD_UPDATE)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that domain
+ * has any CHECK expressions that are not parallel-mode safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (MaxDomainParallelHazardForModify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not parallel-safe.
+ */
+ if ((commandType == CMD_INSERT || commandType == CMD_UPDATE) &&
+ tupdesc->constr != NULL &&
+ tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *checkExpr = stringToNode(check->ccbin);
+ if (max_parallel_hazard_walker((Node *)checkExpr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * MaxParallelHazardForModify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initialMaxParallelHazard == NULL ||
+ *initialMaxParallelHazard == PROPARALLEL_SAFE ||
+ *initialMaxParallelHazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initialMaxParallelHazard == NULL ?
+ PROPARALLEL_SAFE : *initialMaxParallelHazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (MaxRelParallelHazardForModify(rte->relid, parse->commandType, &context));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..a926fff 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModeForModify(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 7ef8cce..12662eb 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -55,5 +55,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char MaxParallelHazardForModify(Query *parse, const char *initialMaxParallelHazard);
#endif /* CLAUSES_H */
--
1.8.3.1
v6-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v6-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 8e6032e626f4e2b35327dcb5a41142acd3e50868 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 30 Oct 2020 00:56:02 +1100
Subject: [PATCH v6 2/2] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Parallel-unsafe trigger
- Foreign key trigger (RI_TRIGGER_FK)
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features
- Partitioned table with parallel-unsafe partition key expressions or support
functions
Where the above-mentioned target table features are parallel-restricted, rather
than parallel-unsafe, at least parallel SELECT may be utilized.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 4 -
src/backend/access/transam/xact.c | 33 ++++++--
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 59 ++++++++++++++-
src/backend/executor/nodeGather.c | 64 +++++++++++++---
src/backend/executor/nodeModifyTable.c | 44 ++++++++++-
src/backend/optimizer/path/costsize.c | 46 ++++++++++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 129 +++++++++++++++++++++++++++++++-
src/backend/optimizer/plan/setrefs.c | 12 ++-
src/backend/optimizer/util/pathnode.c | 55 ++++----------
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 4 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
17 files changed, 395 insertions(+), 75 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1585861..a52d322 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 6130df1..9146878 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -764,12 +778,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed
+ * if currentCommandIdUsed was already true at the start of the
+ * parallel operation (by way of SetCurrentCommandIdUsed()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision
+ * for communicating this back to the leader. Once currentCommandIdUsed
+ * is set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1020,10 +1038,13 @@ IsInParallelMode(void)
* Prepare for entering parallel mode by assigning a FullTransactionId, to be
* included in the transaction state that is serialized in the parallel DSM.
*/
-void PrepareParallelModeForModify(CmdType commandType)
+void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ (void)GetCurrentCommandId(true);
+
(void)GetCurrentTransactionId();
}
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 74a7960..0969a66 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,13 +1528,15 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
/*
* Supported table-modification commands may require additional steps
* prior to entering parallel mode, such as assigning a FullTransactionId.
*/
if (IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType))
{
- PrepareParallelModeForModify(estate->es_plannedstmt->commandType);
+ PrepareParallelModeForModify(estate->es_plannedstmt->commandType, isParallelModifyLeader);
}
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..39f60af 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Allocate space for each worker's returned "# of tuples processed" count. */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1178,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1414,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1436,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of
+ * the parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1504,13 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /* Report the # of tuples processed during parallel INSERT execution. */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..f8d85bc 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if there
+ * are BEFORE STATEMENT triggers, these must be fired by the leader,
+ * not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause -
+ * no readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -418,14 +446,32 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ /*
+ * If the parallel context has already been destroyed, this
+ * function must have been previously called, so just
+ * return.
+ */
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ bool isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not
+ * by the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 29e07b7..220d408 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1833,6 +1834,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local
+ * plan execution (ModifyTable processing). These will be
+ * fired at the end of Gather processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2158,7 +2192,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2235,7 +2273,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index f1dfdc1..23676c1 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a task
+ * for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 94280a7..5893051 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 93c5b6f..5925363 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -1807,7 +1807,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1855,6 +1856,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2391,13 +2393,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving paths, with
+ * Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel INSERT/UPDATE/DELETE,
+ * it seems resonable to use the same number of workers as estimated
+ * for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2414,6 +2505,12 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7583,7 +7680,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification command,
+ * because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 127ea3d..3eec4ea 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,16 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5281a2f..30dc022 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,21 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach (lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *)lfirst(lc);
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3589,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index a926fff..f43a844 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModeForModify(CmdType commandType);
+extern void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..afb8a57 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6c0a7d6..c558eef 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1175,8 +1175,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
-
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
* an UPDATE of a partitioned table.
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
--
1.8.3.1
On Fri, Oct 30, 2020 at 6:09 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Oct 27, 2020 at 8:56 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
IIUC, below is code for this workaround:
+MaxRelParallelHazardForModify(Oid relid, + CmdType commandType, + max_parallel_hazard_context *context) +{ + Relation rel; + TupleDesc tupdesc; + int attnum; + + LOCKMODE lockmode = AccessShareLock; + + /* + * It's possible that this relation is locked for exclusive access + * in another concurrent transaction (e.g. as a result of a + * ALTER TABLE ... operation) until that transaction completes. + * If a share-lock can't be acquired on it now, we have to assume this + * could be the worst-case, so to avoid blocking here until that + * transaction completes, conditionally try to acquire the lock and + * assume and return UNSAFE on failure. + */ + if (ConditionalLockRelationOid(relid, lockmode)) + { + rel = table_open(relid, NoLock); + } + else + { + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + }Do we need this workaround if we lock just the parent table instead of
locking all the tables? Basically, can we safely identify the
parallel-safety of partitioned relation if we just have a lock on
parent relation?I believe the workaround is still needed in this case, because the
workaround was added because of a test in which the parent table was
exclusively locked in another concurrent transaction (as a result of
ALTER TABLE ... ATTACH PARTITION ...) so we could not even get a
ShareLock on the parent table without hanging (and then ending up
failing the test because of it).
Don't you think the test case design is flawed in that case? Because
even simple "select * from tpart;" will hang in planner while taking
share lock (the code flow is:
add_other_rels_to_query->expand_inherited_rtentry->expand_partitioned_rtentry)
once you take exclusive lock for a parallel session on the table.
Currently we never need to acquire any lock for Inserts in the planner
but not sure we can design a test case based on that assumption as we
can see it fails in this basic case.
So at the moment the workaround is needed, even if just trying to lock
the parent table.
I am not convinced, rather I think that the test case is not well
designed unless there is any other way (without taking share lock on
the relation) to determine parallel-safety of Inserts which neither of
us have thought of. I understand that you don't want to change that
test case as part of this patch so you are using this workaround.
I'll do some more testing to determine the secondary issue of whether
locks on the partition tables are needed, but at the moment I believe
they are.
Fair enough but lets determine that by some testing and analysis. I
feel we should even add a comment if we require to lock all partition
tables. I see that we are already doing it for SELECT in the above
mentioned code path so maybe it is okay to do so for Inserts as well.
One more thing I have noticed is that for scan
relations (Select query), we do such checks much later based on
RelOptInfo (see set_rel_consider_parallel) which seems to have most of
the information required to perform parallel-safety checks but I guess
for ModifyTable (aka the Insert table) the equivalent doesn't seem
feasible but have you thought of doing at the later stage in planner?Yes, and in fact I tried putting the checks in a later stage of the
planner, and it's almost successful, except it then makes setting
"parallelModeNeeded" very tricky indeed, because that is expected to
be set based on whether the SQL is safe to run in parallel mode
(paralleModeOK == true) and whether force_parallel_mode is not off.
With parallel safety checks delayed to a later stage in the planner,
it's then not known whether there are certain types of parallel-unsafe
INSERTs (such as INSERT INTO ... VALUES ... ON CONFLICT DO UPDATE
...), because processing for those doesn't reach those later stages of
the planner where parallelism is being considered.
I guess if that is the only case then you can have that check in the
earlier stage of planner (we should be able to do that as the
information is present in Query) and other checks in the later stage.
However, I guess that is not the only case, we need to determine
parallel-safety of index expressions, trigger functions if any, any
other CHECK expressions on each of attribute, etc.
So then to avoid
errors from when parallel-mode is forced on and such unsafe INSERTs
are run, the only real choice is to only allow parallelModeNeeded to
be true for SELECT only (not INSERT), and this is kind of cheating and
also not picking up cases where parallel-safe INSERT is run but
invokes parallel-mode-unsafe features.
My conclusion, at least for the moment, is to leave the check where it is.
Okay, then can we integrate the functionality of
MaxParallelHazardForModify in max_parallel_hazard? Calling it
separately looks bit awkward.
Few other comments on latest patch: =============================== 1. MaxRelParallelHazardForModify() { .. + if (commandType == CMD_INSERT || commandType == CMD_UPDATE) + { + /* ..Why to check CMD_UPDATE here?
That was a bit of forward-thinking, for when/if UPDATE/DELETE is
supported in parallel-mode.
Column default expressions and check-constraints are only applicable
to INSERT and UPDATE.
Note however that currently this function can only ever be called with
commandType == CMD_INSERT.
I feel then for other command types there should be an Assert rather
than try to handle something which is not yet implemented nor it is
clear what all is required for that. It confuses the reader, at least
it confused me. Probably we can write a comment but I don't think we
should have any check for Update at this stage of work.
2. +void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader) +{ + Assert(!IsInParallelMode()); + + if (isParallelModifyLeader) + (void)GetCurrentCommandId(true); + + (void)GetCurrentFullTransactionId();Here, we should use GetCurrentTransactionId() similar to heap_insert
or other heap operations. I am not sure why you have used
GetCurrentFullTransactionId?GetCurrentTransactionId() and GetCurrentFullTransactionId() actually
have the same functionality, just a different return value (which is
not being used here).
Sure but lets use what is required.
But anyway I've changed it to use GetCurrentTransactionId().
But comments in ExecutePlan and PrepareParallelModeForModify still
refer to FullTransactionId.
4. Have you checked the overhead of this on the planner for different
kinds of statements like inserts into tables having 100 or 500
partitions? Similarly, it is good to check the overhead of domain
related checks added in the patch.Checking that now and will post results soon.
Thanks.
--
With Regards,
Amit Kapila.
On Fri, Oct 30, 2020 at 5:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
So then to avoid
errors from when parallel-mode is forced on and such unsafe INSERTs
are run, the only real choice is to only allow parallelModeNeeded to
be true for SELECT only (not INSERT), and this is kind of cheating and
also not picking up cases where parallel-safe INSERT is run but
invokes parallel-mode-unsafe features.
My conclusion, at least for the moment, is to leave the check where it is.Okay, then can we integrate the functionality of
MaxParallelHazardForModify in max_parallel_hazard? Calling it
separately looks bit awkward.
Looking into that.
Few other comments on latest patch: =============================== 1. MaxRelParallelHazardForModify() { .. + if (commandType == CMD_INSERT || commandType == CMD_UPDATE) + { + /* ..Why to check CMD_UPDATE here?
That was a bit of forward-thinking, for when/if UPDATE/DELETE is
supported in parallel-mode.
Column default expressions and check-constraints are only applicable
to INSERT and UPDATE.
Note however that currently this function can only ever be called with
commandType == CMD_INSERT.I feel then for other command types there should be an Assert rather
than try to handle something which is not yet implemented nor it is
clear what all is required for that. It confuses the reader, at least
it confused me. Probably we can write a comment but I don't think we
should have any check for Update at this stage of work.
OK, for now I'll restrict the checks to INSERT, but I'll add comments
to assist with potential future UPDATE support.
2. +void PrepareParallelModeForModify(CmdType commandType, bool isParallelModifyLeader) +{ + Assert(!IsInParallelMode()); + + if (isParallelModifyLeader) + (void)GetCurrentCommandId(true); + + (void)GetCurrentFullTransactionId();Here, we should use GetCurrentTransactionId() similar to heap_insert
or other heap operations. I am not sure why you have used
GetCurrentFullTransactionId?GetCurrentTransactionId() and GetCurrentFullTransactionId() actually
have the same functionality, just a different return value (which is
not being used here).Sure but lets use what is required.
But anyway I've changed it to use GetCurrentTransactionId().
But comments in ExecutePlan and PrepareParallelModeForModify still
refer to FullTransactionId.
I believe those comments are technically correct.
GetCurrentTransactionId() calls AssignTransactionId() to do all the
work - and the comment for that function says "Assigns a new permanent
FullTransactionId to the given TransactionState".
4. Have you checked the overhead of this on the planner for different
kinds of statements like inserts into tables having 100 or 500
partitions? Similarly, it is good to check the overhead of domain
related checks added in the patch.Checking that now and will post results soon.
I am seeing a fair bit of overhead in the planning for the INSERT
parallel-safety checks (mind you, compared to the overall performance
gain, it's not too bad).
Some representative timings for a parallel INSERT of a millions rows
into 100,250 and 500 partitions are shown below.
(1) Without patch
# Partitions Planning Time (ms)
Execution Time (ms)
100 1.014
4176.435
250 0.404
3842.414
500 0.529
4440.633
(2) With Parallel INSERT patch
# Partitions Planning Time (ms)
Execution Time (ms)
100 11.420
2131.148
250 23.269
3472.259
500 36.531
3238.868
I'm looking into how this can be improved by better integration into
the current code, and addressing locking concerns that you've
previously mentioned.
Regards,
Greg Nancarrow
Fujitsu Australia
See attached patches.
Thanks for providing the patches.
I had reviewed
v6-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch, please find
my comments:
-> commandType is not used, we can remove it.
+ * Prepare for entering parallel mode by assigning a FullTransactionId, to
be
+ * included in the transaction state that is serialized in the parallel
DSM.
+ */
+void PrepareParallelModeForModify(CmdType commandType)
+{
+ Assert(!IsInParallelMode());
+
+ (void)GetCurrentTransactionId();
+}
-> As we support insertion of data from the workers, this comments "but as
of now, only the leader backend writes into a completely new table. In the
future, we can extend it to allow workers to write into the table" must be
updated accordingly:
+ * modify any data using a CTE, or if this is a cursor operation,
or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
+ * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and
CREATE
* MATERIALIZED VIEW to use parallel plans, but as of now, only the
leader
* backend writes into a completely new table. In the future, we
can
* extend it to allow workers to write into the table. However, to
allow
-> Also should we specify insert as "insert into select"
-> We could include a small writeup of the design may be in the commit
message. It will be useful for review.
-> I felt the below two assignment statements can be in the else condition:
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard !=
PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in
order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in
parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard =
MaxParallelHazardForModify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard !=
PROPARALLEL_UNSAFE);
+ }
something like:
/*
* Additional parallel-mode safety checks are required in order to
* allow an underlying parallel query to be used for a
* table-modification command that is supported in parallel-mode.
*/
if (glob->parallelModeOK &&
IsModifySupportedInParallelMode(parse->commandType))
glob->maxParallelHazard = MaxParallelHazardForModify(parse,
&glob->maxParallelHazard);
else
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
-> Comments need slight adjustment, maybe you could run pgindent for the
modified code.
+ /*
+ * Supported table-modification commands may require
additional steps
+ * prior to entering parallel mode, such as assigning a
FullTransactionId.
+ */
-> In the below, max_parallel_hazard_test will return true for
PROPARALLEL_RESTRICTED also, Is break intentional in that case? As in case
of RI_TRIGGER_FK for PROPARALLEL_RESTRICTED we continue.
+ if
(max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a
FK exists in
+ * the relation, and this would result in creation of new
CommandIds
+ * on insert/update/delete and this isn't supported in a
parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+ /*
+ * As we're looking for the max parallel hazard, we
don't break
+ * here; examine any further triggers ...
+ */
+ }
-> Should we switch to non-parallel mode in this case, instead of throwing
error?
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin,
&isnull);
+ if (isnull)
+ elog(ERROR, "null conbin for constraint
%u", con->oid);
+ conbin = TextDatumGetCString(val);
-> We could include a few tests for this in regression.
-> We might need some documentation update like in
parallel-query.html/parallel-plans.html, etc
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
Hi Vignesh,
Thanks for reviewing the patches.
On Tue, Nov 3, 2020 at 5:25 PM vignesh C <vignesh21@gmail.com> wrote:
-> commandType is not used, we can remove it. + * Prepare for entering parallel mode by assigning a FullTransactionId, to be + * included in the transaction state that is serialized in the parallel DSM. + */ +void PrepareParallelModeForModify(CmdType commandType) +{ + Assert(!IsInParallelMode()); + + (void)GetCurrentTransactionId(); +}
Thanks, at least for INSERT, it's not needed, so I'll remove it.
-> As we support insertion of data from the workers, this comments "but as of now, only the leader backend writes into a completely new table. In the future, we can extend it to allow workers to write into the table" must be updated accordingly: + * modify any data using a CTE, or if this is a cursor operation, or if + * GUCs are set to values that don't permit parallelism, or if + * parallel-unsafe functions are present in the query tree. * - * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE + * (Note that we do allow CREATE TABLE AS, INSERT, SELECT INTO, and CREATE * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader * backend writes into a completely new table. In the future, we can * extend it to allow workers to write into the table. However, to allow-> Also should we specify insert as "insert into select"
I'll update it, appropriate to each patch.
-> We could include a small writeup of the design may be in the commit message. It will be useful for review.
Will do so for the next patch version.
-> I felt the below two assignment statements can be in the else condition: glob->maxParallelHazard = max_parallel_hazard(parse); glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + + /* + * Additional parallel-mode safety checks are required in order to + * allow an underlying parallel query to be used for a + * table-modification command that is supported in parallel-mode. + */ + if (glob->parallelModeOK && + IsModifySupportedInParallelMode(parse->commandType)) + { + glob->maxParallelHazard = MaxParallelHazardForModify(parse, &glob->maxParallelHazard); + glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + }something like:
/*
* Additional parallel-mode safety checks are required in order to
* allow an underlying parallel query to be used for a
* table-modification command that is supported in parallel-mode.
*/
if (glob->parallelModeOK &&
IsModifySupportedInParallelMode(parse->commandType))
glob->maxParallelHazard = MaxParallelHazardForModify(parse, &glob->maxParallelHazard);
else
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
That won't work. As the comment is trying to point out, additional
parallel-safety checks (i.e. in addition to those done by
max_parallel_hazard()) are required to determine if INSERT can be
safely run in parallel-mode with an underlying parallel query.
Also, the max_parallel_hazard found from first calling
max_parallel_hazard() then needs to be fed into
MaxParallelHazardForModify(), in case it finds a worse parallel
hazard.
For example, max_parallel_hazard() may find something parallel
RESTRICTED, but then the additional parallel-safety checks done by
MaxParallelHazardForModify() find something parallel UNSAFE.
-> Comments need slight adjustment, maybe you could run pgindent for the modified code. + /* + * Supported table-modification commands may require additional steps + * prior to entering parallel mode, such as assigning a FullTransactionId. + */
OK, will run pgindent.
-> In the below, max_parallel_hazard_test will return true for PROPARALLEL_RESTRICTED also, Is break intentional in that case? As in case of RI_TRIGGER_FK for PROPARALLEL_RESTRICTED we continue. + if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context)) + break; + + /* + * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in + * the relation, and this would result in creation of new CommandIds + * on insert/update/delete and this isn't supported in a parallel + * worker (but is safe in the parallel leader). + */ + trigtype = RI_FKey_trigger_type(trigger->tgfoid); + if (trigtype == RI_TRIGGER_FK) + { + context->max_hazard = PROPARALLEL_RESTRICTED; + /* + * As we're looking for the max parallel hazard, we don't break + * here; examine any further triggers ... + */ + }
max_parallel_hazard_test won't return true for PROPARALLEL_RESTRICTED.
max_parallel_hazard_test only returns true when
"context.max_interesting" is found, and that is set to
PROPARALLEL_UNSAFE in max_parallel_hazard_for_modify().
-> Should we switch to non-parallel mode in this case, instead of throwing error? + val = SysCacheGetAttr(CONSTROID, tup, + Anum_pg_constraint_conbin, &isnull); + if (isnull) + elog(ERROR, "null conbin for constraint %u", con->oid); + conbin = TextDatumGetCString(val);
I didn't invent that error check, it's found in several other places
in the Postgres code (that error should only ever occur if the
database has been corrupted or intentionally invalidated).
Having said that, I agree that perhaps it's best to switch to
non-parallel mode in this case, but this wouldn't stop it erroring out
when the plan is actually run.
-> We could include a few tests for this in regression.
Looking at adding relevant test cases.
-> We might need some documentation update like in parallel-query.html/parallel-plans.html, etc
Looking at doc updates.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Nov 4, 2020 at 6:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Nov 3, 2020 at 5:25 PM vignesh C <vignesh21@gmail.com> wrote:
-> commandType is not used, we can remove it. + * Prepare for entering parallel mode by assigning a FullTransactionId, to be + * included in the transaction state that is serialized in the parallel DSM. + */ +void PrepareParallelModeForModify(CmdType commandType) +{ + Assert(!IsInParallelMode()); + + (void)GetCurrentTransactionId(); +}Thanks, at least for INSERT, it's not needed, so I'll remove it.
Or you might want to consider moving the check related to
IsModifySupportedInParallelMode() inside
PrepareParallelModeForModify(). That way the code might look a bit
cleaner.
--
With Regards,
Amit Kapila.
On Wed, Nov 4, 2020 at 2:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Or you might want to consider moving the check related to
IsModifySupportedInParallelMode() inside
PrepareParallelModeForModify(). That way the code might look a bit
cleaner.
Posting an updated Parallel SELECT for "INSERT INTO ... SELECT ..."
patch which addresses previously-identified issues and suggestions,
and adds some tests and doc updates.
I won't post an updated Parallel INSERT patch just yet (which just
builds on the 1st patch), because there's at least a couple of issues
in this 1st patch which need to be discussed first.
Firstly, in order to perform parallel-safety checks in the case of
partitions, the patch currently recursively locks/unlocks
(AccessShareLock) each partition during such checks (as each partition
may itself be a partitioned table). Is there a better way of
performing the parallel-safety checks and reducing the locking
requirements?
Secondly, I found that when running "make check-world", the
"partition-concurrent-attach" test fails, because it is expecting a
partition constraint to be violated on insert, while an "alter table
attach partition ..." is concurrently being executed in another
transaction. Because of the partition locking done by the patch's
parallel-safety checking code, the insert blocks on the exclusive lock
held by the "alter table" in the other transaction until the
transaction ends, so the insert ends up successfully completing (and
thus fails the test) when the other transaction ends. To overcome this
test failure, the patch code was updated to instead perform a
conditional lock on the partition, and on failure (i.e. because of an
exclusive lock held somewhere else), just assume it's parallel-unsafe
because the parallel-safety can't be determined without blocking on
the lock. This is not ideal, but I'm not sure of what other approach
could be used and I am somewhat reluctant to change that test. If
anybody is familiar with the "partition-concurrent-attach" test, any
ideas or insights would be appreciated.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v7-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v7-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From b1cf10102ce35427d22049179ab13eaefa74b83b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 12 Nov 2020 10:23:47 +1100
Subject: [PATCH v7 1/2] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses ON CONFLICT ... DO UPDATE ...
- Target table is a foreign or temporary table
- Target table has a:
- Parallel-unsafe trigger
- Parallel-unsafe index expression
- Parallel-unsafe column default expression
- Parallel-unsafe check constraint
- Partitioned table or partition with any of the above parallel-unsafe features
- Partitioned table with parallel-unsafe partition key expressions or support
functions
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 33 ++-
src/backend/optimizer/util/clauses.c | 436 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 500 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 03c553e..d8c2837 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..2bb74d4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 986d7a5..0434d90 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -318,16 +318,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -336,7 +336,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -344,6 +345,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 85ef873..036723f 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -157,6 +169,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static char domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static char rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -1073,6 +1094,421 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+
+ /*
+ * As we're looking for the max parallel hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ if (ConditionalLockRelationOid(index_oid, lockmode))
+ {
+ index_rel = index_open(index_oid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ if (index_expr_item == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context) == PROPARALLEL_UNSAFE)
+ {
+ index_close(index_rel, lockmode);
+ return context->max_hazard;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ if (lockmode == NoLock)
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ /*
+ * It's possible that this relation is locked for exclusive access in
+ * another concurrent transaction (e.g. as a result of a ALTER TABLE
+ * ... operation) until that transaction completes. If a share-lock
+ * can't be acquired on it now, we have to assume this could be the
+ * worst-case, so to avoid blocking here until that transaction
+ * completes, conditionally try to acquire the lock and assume and
+ * return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ trigger_max_parallel_hazard_for_modify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == NULL ||
+ *initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ *initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == NULL ?
+ PROPARALLEL_SAFE : *initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..591672c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 7ef8cce..a15df99 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -55,5 +55,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
1.8.3.1
v7-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v7-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 1e37513c4c6fe27708b7a98ba59ca1c7ff2a908e Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 13 Nov 2020 15:25:25 +1100
Subject: [PATCH v7 2/2] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 922 ++++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 478 +++++++++++++
5 files changed, 1405 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..938d51a 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..2e117ae
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,922 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ae89ed7..4fa4b97 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -88,6 +88,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 525bdc8..261cab7 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..7d097bc
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,478 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
1.8.3.1
On Fri, Nov 13, 2020 at 8:14 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Nov 4, 2020 at 2:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Or you might want to consider moving the check related to
IsModifySupportedInParallelMode() inside
PrepareParallelModeForModify(). That way the code might look a bit
cleaner.Posting an updated Parallel SELECT for "INSERT INTO ... SELECT ..."
patch which addresses previously-identified issues and suggestions,
and adds some tests and doc updates.
I won't post an updated Parallel INSERT patch just yet (which just
builds on the 1st patch), because there's at least a couple of issues
in this 1st patch which need to be discussed first.Firstly, in order to perform parallel-safety checks in the case of
partitions, the patch currently recursively locks/unlocks
(AccessShareLock) each partition during such checks (as each partition
may itself be a partitioned table). Is there a better way of
performing the parallel-safety checks and reducing the locking
requirements?Secondly, I found that when running "make check-world", the
"partition-concurrent-attach" test fails, because it is expecting a
partition constraint to be violated on insert, while an "alter table
attach partition ..." is concurrently being executed in another
transaction. Because of the partition locking done by the patch's
parallel-safety checking code, the insert blocks on the exclusive lock
held by the "alter table" in the other transaction until the
transaction ends, so the insert ends up successfully completing (and
thus fails the test) when the other transaction ends. To overcome this
test failure, the patch code was updated to instead perform a
conditional lock on the partition, and on failure (i.e. because of an
exclusive lock held somewhere else), just assume it's parallel-unsafe
because the parallel-safety can't be determined without blocking on
the lock. This is not ideal, but I'm not sure of what other approach
could be used and I am somewhat reluctant to change that test. If
anybody is familiar with the "partition-concurrent-attach" test, any
ideas or insights would be appreciated.
Posting an updated set of patches, with some additional testing and
documentation updates, and including the latest version of the
Parallel Insert patch.
Any feedback appreciated, especially on the two points mentioned in
the previous post.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v8-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v8-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 2937b6d1f6a50d285b6f7615b760f007bbb52ac2 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 19 Nov 2020 13:06:16 +1100
Subject: [PATCH v8 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1063 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 526 ++++++++++++
5 files changed, 1594 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..938d51a 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..5f0a89f
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1063 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ae89ed7..4fa4b97 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -88,6 +88,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 525bdc8..261cab7 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..9447120
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,526 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
1.8.3.1
v8-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v8-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 83afb5f82fe9cf046762bd568f1a91c05042e95b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 18:03:21 +1100
Subject: [PATCH v8 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 4 -
src/backend/access/transam/xact.c | 33 ++++++--
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 65 ++++++++++++++-
src/backend/executor/nodeGather.c | 67 ++++++++++++---
src/backend/executor/nodeModifyTable.c | 44 +++++++++-
src/backend/optimizer/path/costsize.c | 46 +++++++++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 140 ++++++++++++++++++++++++++++++--
src/backend/optimizer/plan/setrefs.c | 13 ++-
src/backend/optimizer/util/pathnode.c | 56 ++++---------
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 416 insertions(+), 84 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1b2f704..619318a 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index d8c2837..ef1bba7 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -518,6 +518,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,15 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
if (IsModifySupportedInParallelMode(commandType))
{
+ if (isParallelModifyLeader)
+ (void) GetCurrentCommandId(true);
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2bb74d4..d277e02 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..2f3a75b 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1181,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1417,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1507,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..4217b7d 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isParallelModifyWithReturning)
return slot;
/*
@@ -418,14 +446,33 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ bool isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 29e07b7..3177248 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1833,6 +1834,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2158,7 +2192,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2235,7 +2273,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index f1dfdc1..bbbab3f 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 40abe6f..b08db29 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 0434d90..13de434 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -322,12 +322,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* GUCs are set to values that don't permit parallelism, or if
* parallel-unsafe functions are present in the query tree.
*
+ *
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1807,7 +1809,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1855,6 +1858,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2391,13 +2395,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems resonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2414,6 +2507,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7583,7 +7683,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 127ea3d..c0e6a62 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5281a2f..3ef0fd5 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 591672c..8bd3506 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelMode(CmdType commandType);
+extern void PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..0534544 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 6c0a7d6..88b99fd 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1175,7 +1175,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 6141654..fafa087 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 715a24a..2d08f0c 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c5ffea4..be8f1a0 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -575,15 +575,14 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
/*
* RELATION_IS_OTHER_TEMP
--
1.8.3.1
v8-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v8-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From e43a71f49ea3cb3ab88c0bb7dbcd3d126e72f531 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 17:46:01 +1100
Subject: [PATCH v8 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++++++-------------
src/test/regress/sql/insert_parallel.sql | 20 ++---
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 938d51a..3cc029c 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exeption is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5f0a89f..5755fc1 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -70,14 +70,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -105,7 +105,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -190,9 +190,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -229,9 +229,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -271,9 +271,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -321,9 +321,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -356,9 +356,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -392,9 +392,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -430,9 +430,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -470,9 +470,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -517,21 +517,21 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names4
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names4
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -560,7 +560,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -610,14 +610,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -666,7 +666,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -695,14 +695,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -759,9 +759,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -811,9 +811,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -850,7 +850,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -871,11 +871,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -916,7 +916,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -957,9 +957,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 9447120..ab231cb 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -89,7 +89,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -98,7 +98,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -229,14 +229,14 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -244,7 +244,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -269,7 +269,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -285,7 +285,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -293,7 +293,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -370,7 +370,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -420,7 +420,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
1.8.3.1
v8-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v8-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From bb50d45fc0a609b9f13c96ab7e9b7c6261e5c48c Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 12:17:10 +1100
Subject: [PATCH v8 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 33 ++-
src/backend/optimizer/util/clauses.c | 436 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 500 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 03c553e..d8c2837 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..2bb74d4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 986d7a5..0434d90 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -318,16 +318,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -336,7 +336,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -344,6 +345,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 85ef873..036723f 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -157,6 +169,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static char domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static char rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -1073,6 +1094,421 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+
+ /*
+ * As we're looking for the max parallel hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ if (ConditionalLockRelationOid(index_oid, lockmode))
+ {
+ index_rel = index_open(index_oid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ if (index_expr_item == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context) == PROPARALLEL_UNSAFE)
+ {
+ index_close(index_rel, lockmode);
+ return context->max_hazard;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ if (lockmode == NoLock)
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ /*
+ * It's possible that this relation is locked for exclusive access in
+ * another concurrent transaction (e.g. as a result of a ALTER TABLE
+ * ... operation) until that transaction completes. If a share-lock
+ * can't be acquired on it now, we have to assume this could be the
+ * worst-case, so to avoid blocking here until that transaction
+ * completes, conditionally try to acquire the lock and assume and
+ * return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ trigger_max_parallel_hazard_for_modify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == NULL ||
+ *initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ *initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == NULL ?
+ PROPARALLEL_SAFE : *initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..591672c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 7ef8cce..a15df99 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -55,5 +55,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
1.8.3.1
On Fri, Nov 20, 2020 at 7:44 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches, with some additional testing and
documentation updates, and including the latest version of the
Parallel Insert patch.
Any feedback appreciated, especially on the two points mentioned in
the previous post.
Posting an updated set of patches, since a minor bug was found in the
1st patch that was causing a postgresql-cfbot build failure.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 0be881f06d87f1a3e65ebe085ccdf3fa845a4042 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Mon, 7 Dec 2020 17:40:28 +1100
Subject: [PATCH v9 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 33 ++-
src/backend/optimizer/util/clauses.c | 436 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 500 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 9cd0b7c..0c28a46 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..2bb74d4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1a94b58..00e7388 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 587d494..5b57786 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -147,6 +159,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static char domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static char rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -785,6 +806,421 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+
+ /*
+ * As we're looking for the max parallel hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ if (ConditionalLockRelationOid(index_oid, lockmode))
+ {
+ index_rel = index_open(index_oid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ if (index_expr_item == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ index_close(index_rel, lockmode);
+ return context->max_hazard;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ if (lockmode == NoLock)
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ /*
+ * It's possible that this relation is locked for exclusive access in
+ * another concurrent transaction (e.g. as a result of a ALTER TABLE
+ * ... operation) until that transaction completes. If a share-lock
+ * can't be acquired on it now, we have to assume this could be the
+ * worst-case, so to avoid blocking here until that transaction
+ * completes, conditionally try to acquire the lock and assume and
+ * return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ trigger_max_parallel_hazard_for_modify(rel->trigdesc, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == NULL ||
+ *initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ *initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == NULL ?
+ PROPARALLEL_SAFE : *initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..591672c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 68855d0..34698f7 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
1.8.3.1
v9-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v9-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 2cc7fc53834dd49bd2b631ba528d5f4ec7363004 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 17:46:01 +1100
Subject: [PATCH v9 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++++++-------------
src/test/regress/sql/insert_parallel.sql | 20 ++---
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 938d51a..3cc029c 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exeption is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5f0a89f..5755fc1 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -70,14 +70,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -105,7 +105,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -190,9 +190,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -229,9 +229,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -271,9 +271,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -321,9 +321,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -356,9 +356,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -392,9 +392,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -430,9 +430,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -470,9 +470,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -517,21 +517,21 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names4
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names4
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -560,7 +560,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -610,14 +610,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -666,7 +666,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -695,14 +695,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -759,9 +759,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -811,9 +811,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -850,7 +850,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -871,11 +871,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -916,7 +916,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -957,9 +957,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 9447120..ab231cb 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -89,7 +89,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -98,7 +98,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -229,14 +229,14 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -244,7 +244,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -269,7 +269,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -285,7 +285,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -293,7 +293,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -370,7 +370,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -420,7 +420,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
1.8.3.1
v9-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v9-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From f138dfebad63922f95ccd2bd863183de5a7ca6a6 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 18:03:21 +1100
Subject: [PATCH v9 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 4 -
src/backend/access/transam/xact.c | 33 ++++++--
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 65 ++++++++++++++-
src/backend/executor/nodeGather.c | 67 ++++++++++++---
src/backend/executor/nodeModifyTable.c | 44 +++++++++-
src/backend/optimizer/path/costsize.c | 46 +++++++++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 140 ++++++++++++++++++++++++++++++--
src/backend/optimizer/plan/setrefs.c | 13 ++-
src/backend/optimizer/util/pathnode.c | 56 ++++---------
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 416 insertions(+), 84 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 1b2f704..619318a 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 0c28a46..bad6a12 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -518,6 +518,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,15 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
if (IsModifySupportedInParallelMode(commandType))
{
+ if (isParallelModifyLeader)
+ (void) GetCurrentCommandId(true);
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2bb74d4..d277e02 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..2f3a75b 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1181,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1417,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1507,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..4217b7d 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isParallelModifyWithReturning)
return slot;
/*
@@ -418,14 +446,33 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ bool isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index e0f2428..58bf07c 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1830,6 +1831,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2155,7 +2189,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2232,7 +2270,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 22d6935..52ecb3a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 40abe6f..b08db29 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 00e7388..81d70e1 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -320,12 +320,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* GUCs are set to values that don't permit parallelism, or if
* parallel-unsafe functions are present in the query tree.
*
+ *
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1805,7 +1807,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1853,6 +1856,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2378,13 +2382,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems resonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2401,6 +2494,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7570,7 +7670,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 127ea3d..c0e6a62 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5147895..2b55f7f 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 591672c..8bd3506 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelMode(CmdType commandType);
+extern void PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..0534544 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 61ba4c3..e576038 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1164,7 +1164,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 8e621d2..a2ae2f6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 3bd7072..77011bc 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c5ffea4..be8f1a0 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -575,15 +575,14 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
/*
* RELATION_IS_OTHER_TEMP
--
1.8.3.1
v9-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v9-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 6c7658e206e8eb95c975edca647209af264960a3 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 19 Nov 2020 13:06:16 +1100
Subject: [PATCH v9 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1063 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 526 ++++++++++++
5 files changed, 1594 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..938d51a 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..5f0a89f
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1063 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ae89ed7..4fa4b97 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -88,6 +88,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 525bdc8..261cab7 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..9447120
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,526 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
1.8.3.1
On Mon, Dec 7, 2020 at 2:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Nov 20, 2020 at 7:44 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches, with some additional testing and
documentation updates, and including the latest version of the
Parallel Insert patch.
Any feedback appreciated, especially on the two points mentioned in
the previous post.Posting an updated set of patches, since a minor bug was found in the
1st patch that was causing a postgresql-cfbot build failure.
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.
Few comments:
1)
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ context->max_hazard = PROPARALLEL_RESTRICTED;
+
+ /*
+ * As we're looking for the max parallel
hazard, we don't break
+ * here; examine any further triggers ...
+ */
+ }
Can we change this something like:
trigtype = RI_FKey_trigger_type(trigger->tgfoid);
if (trigtype == RI_TRIGGER_FK)
{
if(max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)
break;
}
This below line is not required as it will be taken care by
max_parallel_hazard_test.
context->max_hazard = PROPARALLEL_RESTRICTED;
2)
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+
command_type,
+
context,
+
AccessShareLock) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
Can we change this something like:
/* Recursively check each partition ... */
pdesc = RelationGetPartitionDesc(rel);
for (i = 0; i < pdesc->nparts; i++)
{
char max_hazard = rel_max_parallel_hazard_for_modify(pdesc->oids[i],
command_type,
context,
AccessShareLock);
if(max_parallel_hazard_test(max_hazard, context)
{
table_close(rel, lockmode);
return context->max_hazard;
}
}
3)
Similarly for the below:
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context) ==
PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL &&
+ trigger_max_parallel_hazard_for_modify(rel->trigdesc,
context) == PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
4) Similar change required for the below:
+ /*
+ * If the column is of a DOMAIN type,
determine whether that
+ * domain has any CHECK expressions that are
not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if
(domain_max_parallel_hazard_for_modify(att->atttypid, context) ==
PROPARALLEL_UNSAFE)
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.
Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.
I've attached a new set of patches that includes your suggested improvements.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v10-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v10-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From e1bf0936bf46fb1c844f8052c6a5285c447ac72a Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 19 Nov 2020 13:06:16 +1100
Subject: [PATCH v10 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1063 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 526 ++++++++++++
5 files changed, 1594 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..938d51a 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..5f0a89f
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1063 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ae89ed7..4fa4b97 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -88,6 +88,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 525bdc8..261cab7 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..9447120
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,526 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
1.8.3.1
v10-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v10-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From cefb0fca4aa28bbb982bd49edd33fecf3b4450d4 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 17:46:01 +1100
Subject: [PATCH v10 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++++++-------------
src/test/regress/sql/insert_parallel.sql | 20 ++---
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 938d51a..3cc029c 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exeption is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5f0a89f..5755fc1 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -70,14 +70,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -105,7 +105,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -190,9 +190,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -229,9 +229,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -271,9 +271,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -321,9 +321,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -356,9 +356,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -392,9 +392,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -430,9 +430,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -470,9 +470,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -517,21 +517,21 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names4
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names4
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -560,7 +560,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -610,14 +610,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -666,7 +666,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -695,14 +695,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -759,9 +759,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -811,9 +811,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -850,7 +850,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -871,11 +871,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -916,7 +916,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -957,9 +957,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 9447120..ab231cb 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -89,7 +89,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -98,7 +98,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -229,14 +229,14 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -244,7 +244,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -269,7 +269,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -285,7 +285,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -293,7 +293,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -370,7 +370,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -420,7 +420,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
1.8.3.1
v10-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v10-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From f5f2aa115bbbf832460ce08a71bc15634511a05a Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 9 Dec 2020 14:42:27 +1100
Subject: [PATCH v10 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 33 ++-
src/backend/optimizer/util/clauses.c | 439 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 503 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 9cd0b7c..0c28a46 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..2bb74d4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1a94b58..00e7388 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index cb7fa66..0767267 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -42,7 +48,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -50,6 +60,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -147,6 +159,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static char domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static char rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -785,6 +806,424 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ Trigger *trigger = &trigdesc->triggers[i];
+ int trigtype;
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ break;
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ if (ConditionalLockRelationOid(index_oid, lockmode))
+ {
+ index_rel = index_open(index_oid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ if (index_expr_item == NULL) /* shouldn't happen */
+ elog(ERROR, "too few entries in indexprs list");
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ index_close(index_rel, lockmode);
+ return context->max_hazard;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+ char max_hazard;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ if (lockmode == NoLock)
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ /*
+ * It's possible that this relation is locked for exclusive access in
+ * another concurrent transaction (e.g. as a result of a ALTER TABLE
+ * ... operation) until that transaction completes. If a share-lock
+ * can't be acquired on it now, we have to assume this could be the
+ * worst-case, so to avoid blocking here until that transaction
+ * completes, conditionally try to acquire the lock and assume and
+ * return UNSAFE on failure.
+ */
+ if (ConditionalLockRelationOid(relid, lockmode))
+ {
+ rel = table_open(relid, NoLock);
+ }
+ else
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ max_hazard = rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ max_hazard = index_expr_max_parallel_hazard_for_modify(rel, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ max_hazard = trigger_max_parallel_hazard_for_modify(rel->trigdesc, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ max_hazard = domain_max_parallel_hazard_for_modify(att->atttypid, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == NULL ||
+ *initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ *initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == NULL ?
+ PROPARALLEL_SAFE : *initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..591672c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 68855d0..34698f7 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
1.8.3.1
v10-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v10-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 5ddc643d866f2553127f5fec3b01c2a6122736f9 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 9 Dec 2020 13:10:58 +1100
Subject: [PATCH v10 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 4 -
src/backend/access/transam/xact.c | 33 ++++++--
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 65 ++++++++++++++-
src/backend/executor/nodeGather.c | 69 +++++++++++++---
src/backend/executor/nodeModifyTable.c | 44 +++++++++-
src/backend/optimizer/path/costsize.c | 46 +++++++++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 140 ++++++++++++++++++++++++++++++--
src/backend/optimizer/plan/setrefs.c | 13 ++-
src/backend/optimizer/util/pathnode.c | 56 ++++---------
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 418 insertions(+), 84 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index a9583f3..444fc37 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,6 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 0c28a46..bad6a12 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -518,6 +518,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,15 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
if (IsModifySupportedInParallelMode(commandType))
{
+ if (isParallelModifyLeader)
+ (void) GetCurrentCommandId(true);
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2bb74d4..d277e02 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..2f3a75b 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1181,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1417,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1507,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..c249195 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isParallelModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isParallelModifyLeader;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index e0f2428..58bf07c 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1830,6 +1831,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2155,7 +2189,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2232,7 +2270,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 22d6935..52ecb3a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 40abe6f..b08db29 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 00e7388..81d70e1 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -320,12 +320,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* GUCs are set to values that don't permit parallelism, or if
* parallel-unsafe functions are present in the query tree.
*
+ *
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1805,7 +1807,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1853,6 +1856,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2378,13 +2382,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems resonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2401,6 +2494,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7570,7 +7670,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 127ea3d..c0e6a62 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5147895..2b55f7f 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 591672c..8bd3506 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelMode(CmdType commandType);
+extern void PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..0534544 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 61ba4c3..e576038 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1164,7 +1164,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 8e621d2..a2ae2f6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 3bd7072..77011bc 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c5ffea4..be8f1a0 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -575,15 +575,14 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
/*
* RELATION_IS_OTHER_TEMP
--
1.8.3.1
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
I was going through v10-0001 patch where we are parallelizing only the
select part.
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action ==
ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
I understand that we can now allow updates from the worker, but what
is the problem if we allow the parallel select even if there is an
update in the leader?
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Dec 9, 2020 at 2:38 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
I was going through v10-0001 patch where we are parallelizing only the
select part.+ /* + * UPDATE is not currently supported in parallel-mode, so prohibit + * INSERT...ON CONFLICT...DO UPDATE... + */ + if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE) + return PROPARALLEL_UNSAFE;I understand that we can now allow updates from the worker, but what
is the problem if we allow the parallel select even if there is an
update in the leader?
I think we can't allow update even in leader without having a
mechanism for a shared combocid table. Right now, we share the
ComboCids at the beginning of the parallel query and then never change
it during the parallel query but if we allow updates in the leader
backend which can generate a combocid then we need a mechanism to
propagate that change. Does this make sense?
--
With Regards,
Amit Kapila.
On Wed, Dec 9, 2020 at 4:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 9, 2020 at 2:38 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
I was going through v10-0001 patch where we are parallelizing only the
select part.+ /* + * UPDATE is not currently supported in parallel-mode, so prohibit + * INSERT...ON CONFLICT...DO UPDATE... + */ + if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE) + return PROPARALLEL_UNSAFE;I understand that we can now allow updates from the worker, but what
is the problem if we allow the parallel select even if there is an
update in the leader?I think we can't allow update even in leader without having a
mechanism for a shared combocid table. Right now, we share the
ComboCids at the beginning of the parallel query and then never change
it during the parallel query but if we allow updates in the leader
backend which can generate a combocid then we need a mechanism to
propagate that change. Does this make sense?
Okay, got it. Basically, ONCONFLICT_UPDATE might run inside some
transaction block and there is a possibility that update may try to
update the same tuple is previously inserted by the same transaction
and in that case, it will generate the combo cid. Thanks for
clarifying.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.
In v10-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO,
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2049,10 +2049,6 @@ heap_prepare_insert(Relation relation,
HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
- if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
I think I have given a comment long back that here we can have an
Assert to check if it is a parallel-worker and relation has a
foreign-key and probably other conditions if possible. It is better to
protect such cases from happening due to any bugs. Is there a reason
you have not handled it?
[1]: /messages/by-id/CAA4eK1KyftVDgovvRQmdV1b=nN0R-KqdWZqiu7jZ1GYQ7SO9OA@mail.gmail.com
--
With Regards,
Amit Kapila.
On Wed, Dec 9, 2020 at 4:18 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Dec 9, 2020 at 4:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 9, 2020 at 2:38 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
I was going through v10-0001 patch where we are parallelizing only the
select part.+ /* + * UPDATE is not currently supported in parallel-mode, so prohibit + * INSERT...ON CONFLICT...DO UPDATE... + */ + if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE) + return PROPARALLEL_UNSAFE;I understand that we can now allow updates from the worker, but what
is the problem if we allow the parallel select even if there is an
update in the leader?I think we can't allow update even in leader without having a
mechanism for a shared combocid table. Right now, we share the
ComboCids at the beginning of the parallel query and then never change
it during the parallel query but if we allow updates in the leader
backend which can generate a combocid then we need a mechanism to
propagate that change. Does this make sense?Okay, got it. Basically, ONCONFLICT_UPDATE might run inside some
transaction block and there is a possibility that update may try to
update the same tuple is previously inserted by the same transaction
and in that case, it will generate the combo cid. Thanks for
clarifying.
We can probably add a comment in the patch so that it is clear why we
are not allowing this case.
--
With Regards,
Amit Kapila.
On Wed, 9 Dec 2020 at 5:41 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 9, 2020 at 4:18 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Dec 9, 2020 at 4:03 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
On Wed, Dec 9, 2020 at 2:38 PM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com>
wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com>
wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly makeit
generic by handling like the comments below, that way this
parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with thechanges,
I'm providing these comments so that this patch can be
generalized for
parallel checks and the same can also be used by parallel copy.
Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested
improvements.
I was going through v10-0001 patch where we are parallelizing only
the
select part.
+ /* + * UPDATE is not currently supported in parallel-mode, so prohibit + * INSERT...ON CONFLICT...DO UPDATE... + */ + if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE) + return PROPARALLEL_UNSAFE;I understand that we can now allow updates from the worker, but what
is the problem if we allow the parallel select even if there is an
update in the leader?I think we can't allow update even in leader without having a
mechanism for a shared combocid table. Right now, we share the
ComboCids at the beginning of the parallel query and then never change
it during the parallel query but if we allow updates in the leader
backend which can generate a combocid then we need a mechanism to
propagate that change. Does this make sense?Okay, got it. Basically, ONCONFLICT_UPDATE might run inside some
transaction block and there is a possibility that update may try to
update the same tuple is previously inserted by the same transaction
and in that case, it will generate the combo cid. Thanks for
clarifying.We can probably add a comment in the patch so that it is clear why we
are not allowing this case.
+1
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
From: Greg Nancarrow <gregn4422@gmail.com>
Firstly, in order to perform parallel-safety checks in the case of partitions, the
patch currently recursively locks/unlocks
(AccessShareLock) each partition during such checks (as each partition may
itself be a partitioned table). Is there a better way of performing the
parallel-safety checks and reducing the locking requirements?
First of all, as you demonstrated the planning time and execution time of parallel insert, I think the increased planning time is negligible when the parallel insert is intentionally used for loading large amount of data. However, it's a problem if the overhead is imposed on OLTP transactions. Does the overhead occur with the default values of max_parallel_workers_per_gather = 2 and max_parall_workers = 8?
To avoid this heavy checking during planning, I'm wondering if we can have an attribute in pg_class, something like relhasindexes and relhas triggers. The concerning point is that we have to maintain the accuracy of the value when dropping ancillary objects around the table/partition.
Secondly, I found that when running "make check-world", the
"partition-concurrent-attach" test fails, because it is expecting a partition
constraint to be violated on insert, while an "alter table attach partition ..." is
concurrently being executed in another transaction. Because of the partition
locking done by the patch's parallel-safety checking code, the insert blocks on
the exclusive lock held by the "alter table" in the other transaction until the
transaction ends, so the insert ends up successfully completing (and thus fails
the test) when the other transaction ends. To overcome this test failure, the
patch code was updated to instead perform a conditional lock on the partition,
and on failure (i.e. because of an exclusive lock held somewhere else), just
assume it's parallel-unsafe because the parallel-safety can't be determined
without blocking on the lock. This is not ideal, but I'm not sure of what other
approach could be used and I am somewhat reluctant to change that test. If
anybody is familiar with the "partition-concurrent-attach" test, any ideas or
insights would be appreciated.
That test looks sane. I think what we should do is to disable parallel operation during that test. It looks like some of other existing test cases disable parallel query by setting max_parallel_workers_per_gather to 0. It's not strange that some tests fail with some configuration. autovacuum is disabled in many places of the regression test.
Rather, I don't think we should introduce the trick to use ConditionalLockAcquire(). Otherwise, the insert would be executed in a serial fashion without the user knowing it -- "What? The insert suddenly slowed down multiple times today, and it didn't finish within the planned maintenance window. What's wrong?"
Regards
Takayuki Tsunakawa
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
Thanks for fixing and posting a new patch.
Few comments:
+ Node *index_expr;
+
+ if (index_expr_item == NULL)
/* shouldn't happen */
+ elog(ERROR, "too few
entries in indexprs list");
+
+ index_expr = (Node *)
lfirst(index_expr_item);
We can change this elog to below to maintain consistency:
if (index_expr_item == NULL) /* shouldn't happen */
{
context->max_hazard = PROPARALLEL_UNSAFE;
return context->max_hazard;
}
static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
/*
* To allow parallel inserts, we need to ensure that they are safe to be
* performed in workers. We have the infrastructure to allow parallel
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
I felt we could remove the above comments or maybe rephrase it.
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
On Thu, Dec 10, 2020 at 3:50 PM vignesh C <vignesh21@gmail.com> wrote:
Few comments: + Node *index_expr; + + if (index_expr_item == NULL) /* shouldn't happen */ + elog(ERROR, "too few entries in indexprs list"); + + index_expr = (Node *) lfirst(index_expr_item);We can change this elog to below to maintain consistency:
if (index_expr_item == NULL) /* shouldn't happen */
{
context->max_hazard = PROPARALLEL_UNSAFE;
return context->max_hazard;
}
Thanks. I think you pointed out something similar to this before, but
somehow I must have missed updating this as well (I was just following
existing error handling for this case in the Postgres code).
I'll update it as you suggest, in the next version of the patch I post.
static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
/*
* To allow parallel inserts, we need to ensure that they are safe to be
* performed in workers. We have the infrastructure to allow parallel
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
I felt we could remove the above comments or maybe rephrase it.
That is Amit's comment, and I'm reluctant to change it because it is
still applicable even after application of this patch.
Amit has previously suggested that I add an Assert here, to match the
comment (to replace the original Parallel-worker error-check that I
removed), so I am looking into that.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Dec 9, 2020 at 10:11 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 9, 2020 at 1:35 AM vignesh C <vignesh21@gmail.com> wrote:
Most of the code present in
v9-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch is
applicable for parallel copy patch also. The patch in this thread
handles the check for PROPARALLEL_UNSAFE, we could slightly make it
generic by handling like the comments below, that way this parallel
safety checks can be used based on the value set in
max_parallel_hazard_context. There is nothing wrong with the changes,
I'm providing these comments so that this patch can be generalized for
parallel checks and the same can also be used by parallel copy.Hi Vignesh,
You are absolutely right in pointing that out, the code was taking
short-cuts knowing that for Parallel Insert,
"max_parallel_hazard_context.max_interesting" had been set to
PROPARALLEL_UNSAFE, which doesn't allow that code to be generically
re-used by other callers.I've attached a new set of patches that includes your suggested improvements.
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
Why do we need to serialize the transaction ID for 0001? I mean in
0001 we are just allowing the SELECT to be executed in parallel so why
we would need the transaction Id for that. I agree that we would need
this once we try to perform the Insert also from the worker in the
remaining patches.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Thu, Dec 10, 2020 at 1:23 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
From: Greg Nancarrow <gregn4422@gmail.com>
Firstly, in order to perform parallel-safety checks in the case of partitions, the
patch currently recursively locks/unlocks
(AccessShareLock) each partition during such checks (as each partition may
itself be a partitioned table). Is there a better way of performing the
parallel-safety checks and reducing the locking requirements?First of all, as you demonstrated the planning time and execution time of parallel insert, I think the increased planning time is negligible when the parallel insert is intentionally used for loading large amount of data. However, it's a problem if the overhead is imposed on OLTP transactions. Does the overhead occur with the default values of max_parallel_workers_per_gather = 2 and max_parall_workers = 8?
To avoid this heavy checking during planning, I'm wondering if we can have an attribute in pg_class, something like relhasindexes and relhas triggers. The concerning point is that we have to maintain the accuracy of the value when dropping ancillary objects around the table/partition.
Having information in another table that needs to be accessed is
likely to also have locking requirements.
Here the issue is specifically with partitions, because otherwise if
the target relation is not a partitioned table, it will already be
locked prior to planning as part of the parse/re-write phase (and you
will notice that the initial lock-mode, used by the parallel-safety
checking code for opening the table, is NoLock).
Secondly, I found that when running "make check-world", the
"partition-concurrent-attach" test fails, because it is expecting a partition
constraint to be violated on insert, while an "alter table attach partition ..." is
concurrently being executed in another transaction. Because of the partition
locking done by the patch's parallel-safety checking code, the insert blocks on
the exclusive lock held by the "alter table" in the other transaction until the
transaction ends, so the insert ends up successfully completing (and thus fails
the test) when the other transaction ends. To overcome this test failure, the
patch code was updated to instead perform a conditional lock on the partition,
and on failure (i.e. because of an exclusive lock held somewhere else), just
assume it's parallel-unsafe because the parallel-safety can't be determined
without blocking on the lock. This is not ideal, but I'm not sure of what other
approach could be used and I am somewhat reluctant to change that test. If
anybody is familiar with the "partition-concurrent-attach" test, any ideas or
insights would be appreciated.That test looks sane. I think what we should do is to disable parallel operation during that test. It looks like some of other existing test cases disable parallel query by setting max_parallel_workers_per_gather to 0. It's not strange that some tests fail with some configuration. autovacuum is disabled in many places of the regression test.
Rather, I don't think we should introduce the trick to use ConditionalLockAcquire(). Otherwise, the insert would be executed in a serial fashion without the user knowing it -- "What? The insert suddenly slowed down multiple times today, and it didn't finish within the planned maintenance window. What's wrong?"
I think that's probably the best idea, to disable parallel operation
during that test.
However, that doesn't change the fact that, after removal of that
"trick", then the partition locking used in the parallel-safety
checking code will block, if a concurrent transaction has exclusively
locked that partition (as in this test case), and thus there is no
guarantee that a parallel insert will execute faster compared to
serial execution (as such locks tend to be held until the end of the
transaction).
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Dec 10, 2020 at 5:25 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
/* + * PrepareParallelMode + * + * Prepare for entering parallel mode, based on command-type. + */ +void +PrepareParallelMode(CmdType commandType) +{ + Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF); + + if (IsModifySupportedInParallelMode(commandType)) + { + /* + * Prepare for entering parallel mode by assigning a + * FullTransactionId, to be included in the transaction state that is + * serialized in the parallel DSM. + */ + (void) GetCurrentTransactionId(); + } +}Why do we need to serialize the transaction ID for 0001? I mean in
0001 we are just allowing the SELECT to be executed in parallel so why
we would need the transaction Id for that. I agree that we would need
this once we try to perform the Insert also from the worker in the
remaining patches.
There's a very good reason. It's related to parallel-mode checks for
Insert and how the XID is lazily acquired if required.
When allowing SELECT to be executed in parallel, we're in
parallel-mode and the leader interleaves Inserts with retrieval of the
tuple data from the workers.
You will notice that heap_insert() calls GetTransactionId() as the
very first thing it does. If the FullTransactionId is not valid,
AssignTransactionId() is then called, which then executes this code:
/*
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs at this point.
*/
if (IsInParallelMode() || IsParallelWorker())
elog(ERROR, "cannot assign XIDs during a parallel operation");
So that code (currently) has no way of knowing that a XID is being
(lazily) assigned at the beginning, or somewhere in the middle of, a
parallel operation.
This is the reason why PrepareParallelMode() is calling
GetTransactionId() up-front, to ensure a FullTransactionId is assigned
up-front, prior to parallel-mode (so then there won't be an attempted
XID assignment).
If you remove the GetTransactionId() call from PrepareParallelMode()
and run "make installcheck-world" with "force_parallel_mode=regress"
in effect, many tests will fail with:
ERROR: cannot assign XIDs during a parallel operation
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Dec 10, 2020 at 1:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Dec 10, 2020 at 5:25 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
/* + * PrepareParallelMode + * + * Prepare for entering parallel mode, based on command-type. + */ +void +PrepareParallelMode(CmdType commandType) +{ + Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF); + + if (IsModifySupportedInParallelMode(commandType)) + { + /* + * Prepare for entering parallel mode by assigning a + * FullTransactionId, to be included in the transaction state that is + * serialized in the parallel DSM. + */ + (void) GetCurrentTransactionId(); + } +}Why do we need to serialize the transaction ID for 0001? I mean in
0001 we are just allowing the SELECT to be executed in parallel so why
we would need the transaction Id for that. I agree that we would need
this once we try to perform the Insert also from the worker in the
remaining patches.There's a very good reason. It's related to parallel-mode checks for
Insert and how the XID is lazily acquired if required.
When allowing SELECT to be executed in parallel, we're in
parallel-mode and the leader interleaves Inserts with retrieval of the
tuple data from the workers.
You will notice that heap_insert() calls GetTransactionId() as the
very first thing it does. If the FullTransactionId is not valid,
AssignTransactionId() is then called, which then executes this code:/*
* Workers synchronize transaction state at the beginning of each parallel
* operation, so we can't account for new XIDs at this point.
*/
if (IsInParallelMode() || IsParallelWorker())
elog(ERROR, "cannot assign XIDs during a parallel operation");So that code (currently) has no way of knowing that a XID is being
(lazily) assigned at the beginning, or somewhere in the middle of, a
parallel operation.
This is the reason why PrepareParallelMode() is calling
GetTransactionId() up-front, to ensure a FullTransactionId is assigned
up-front, prior to parallel-mode (so then there won't be an attempted
XID assignment).If you remove the GetTransactionId() call from PrepareParallelMode()
and run "make installcheck-world" with "force_parallel_mode=regress"
in effect, many tests will fail with:
ERROR: cannot assign XIDs during a parallel operation
Yeah got it, I missed that point that the goal is the avoid assigning
the Transaction Id when we are in parallel mode. But IIUC at least
for the first patch we don't want to serialize the XID in the
transaction state right because workers don't need the xid as they are
only doing select. So maybe we can readjust the comment slightly in
the below code
+ * Prepare for entering parallel mode by assigning a + * FullTransactionId, to be included in the transaction state that is + * serialized in the parallel DSM. + */ + (void) GetCurrentTransactionId();
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
Posting an updated set of patches to address recent feedback:
- Removed conditional-locking code used in parallel-safety checking
code (Tsunakawa-san feedback). It turns out that for the problem test
case, no parallel-safety checking should be occurring that locks
relations because those inserts are specifying VALUES, not an
underlying SELECT, so the parallel-safety checking code was updated to
bail out early if no underlying SELECT is specified for the INSERT. No
change to the test code was required.
- Added comment to better explain the reason for treating "INSERT ...
ON CONFLICT ... DO UPDATE" as parallel-unsafe (Dilip)
- Added assertion to heap_prepare_insert() (Amit)
- Updated error handling for NULL index_expr_item case (Vignesh)
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 8ad1eb98bc93c2a3651657c78fbc5e49932ca300 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 11 Dec 2020 19:08:23 +1100
Subject: [PATCH v11 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 33 ++-
src/backend/optimizer/util/clauses.c | 435 +++++++++++++++++++++++++++++++++++
src/include/access/xact.h | 15 ++
src/include/optimizer/clauses.h | 1 +
6 files changed, 499 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 9cd0b7c..0c28a46 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
+
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 7179f58..2bb74d4 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1a94b58..00e7388 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
else
{
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index e3a81a7..8296f7e 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "catalog/index.h"
+#include "catalog/indexing.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,7 +49,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
#include "utils/builtins.h"
@@ -51,6 +61,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -148,6 +160,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static char trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static char index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static char domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static char rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -786,6 +807,420 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static char
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ break;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ break;
+ }
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static char
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ index_rel = index_open(index_oid, lockmode);
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ index_close(index_rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ index_close(index_rel, lockmode);
+ return context->max_hazard;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return context->max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static char
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static char
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+ char max_hazard;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ rel = table_open(relid, lockmode);
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ max_hazard = rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ max_hazard = index_expr_max_parallel_hazard_for_modify(rel, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ max_hazard = trigger_max_parallel_hazard_for_modify(rel->trigdesc, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ max_hazard = domain_max_parallel_hazard_for_modify(att->atttypid, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return context->max_hazard;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ ListCell *lc;
+ bool hasSubQuery;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * If there is no underlying query, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == NULL ||
+ *initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ *initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == NULL ?
+ PROPARALLEL_SAFE : *initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ return (rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock));
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 7320de3..591672c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 68855d0..34698f7 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, const char *initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
1.8.3.1
v11-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v11-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 8c5f5a59e03b82ea32327870476b5209a86c3de9 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 20 Nov 2020 17:46:01 +1100
Subject: [PATCH v11 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++++++-------------
src/test/regress/sql/insert_parallel.sql | 20 ++---
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 938d51a..3cc029c 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exeption is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5f0a89f..5755fc1 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -70,14 +70,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -105,7 +105,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -190,9 +190,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -229,9 +229,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -271,9 +271,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -321,9 +321,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -356,9 +356,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -392,9 +392,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -430,9 +430,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -470,9 +470,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -517,21 +517,21 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names4
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names4
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -560,7 +560,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -610,14 +610,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -666,7 +666,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -695,14 +695,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -759,9 +759,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -811,9 +811,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -850,7 +850,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -871,11 +871,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -916,7 +916,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -957,9 +957,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 9447120..ab231cb 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -89,7 +89,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -98,7 +98,7 @@ select * from para_insert_p1 where unique1 >= 9990 order by unique1;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -229,14 +229,14 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -244,7 +244,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -269,7 +269,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -285,7 +285,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -293,7 +293,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -370,7 +370,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -420,7 +420,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
1.8.3.1
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 1a4a12c50d796916824ea728fe4ab4a1f2625c26 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 11 Dec 2020 17:19:10 +1100
Subject: [PATCH v11 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++++++-
src/backend/access/transam/xact.c | 33 ++++++--
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 65 ++++++++++++++-
src/backend/executor/nodeGather.c | 69 +++++++++++++---
src/backend/executor/nodeModifyTable.c | 44 +++++++++-
src/backend/optimizer/path/costsize.c | 46 +++++++++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 140 ++++++++++++++++++++++++++++++--
src/backend/optimizer/plan/setrefs.c | 13 ++-
src/backend/optimizer/util/pathnode.c | 56 ++++---------
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 445 insertions(+), 83 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index a9583f3..53df895 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2049,10 +2052,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 0c28a46..bad6a12 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -518,6 +518,20 @@ GetCurrentFullTransactionIdIfAny(void)
}
/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
+/*
* MarkCurrentTransactionIdLoggedIfAny
*
* Remember that the current xid - if it is assigned - now has been wal logged.
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,15 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
if (IsModifySupportedInParallelMode(commandType))
{
+ if (isParallelModifyLeader)
+ (void) GetCurrentCommandId(true);
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2bb74d4..d277e02 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index befde52..2f3a75b 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1181,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1417,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1507,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index a01b46a..c249195 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isParallelModifyLeader = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isParallelModifyWithReturning = isParallelModifyLeader && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isParallelModifyLeader)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isParallelModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isParallelModifyLeader;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isParallelModifyLeader = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isParallelModifyLeader)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index e0f2428..58bf07c 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1830,6 +1831,39 @@ fireASTriggers(ModifyTableState *node)
}
/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
+/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
*/
@@ -2155,7 +2189,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2232,7 +2270,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 22d6935..52ecb3a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -213,6 +213,52 @@ clamp_row_est(double nrows)
/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
+/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 40abe6f..b08db29 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 00e7388..81d70e1 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -320,12 +320,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
* GUCs are set to values that don't permit parallelism, or if
* parallel-unsafe functions are present in the query tree.
*
+ *
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1805,7 +1807,8 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
}
/*--------------------
@@ -1853,6 +1856,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ int parallel_modify_partial_path_count = 0;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2378,13 +2382,102 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
returningLists,
rowMarks,
parse->onConflict,
- assign_special_exec_param(root));
+ assign_special_exec_param(root),
+ 0);
}
/* And shove it into final_rel */
add_path(final_rel, path);
}
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ int parallelModifyWorkers;
+
+ /*
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
+ */
+ foreach(lc, current_rel->partial_pathlist)
+ {
+ Path *path = (Path *) lfirst(lc);
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * Add the ModifyTable node.
+ */
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems resonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ root->rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelModifyWorkers);
+
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_count++;
+ }
+ }
+
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
@@ -2401,6 +2494,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7570,7 +7670,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 127ea3d..c0e6a62 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 5147895..2b55f7f 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 591672c..8bd3506 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelMode(CmdType commandType);
+extern void PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 5a39a5b..0534544 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 46a2dc9..e332482 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 61ba4c3..e576038 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1164,7 +1164,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 8e621d2..a2ae2f6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 3bd7072..77011bc 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index c5ffea4..be8f1a0 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -575,15 +575,14 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
/*
* RELATION_IS_OTHER_TEMP
--
1.8.3.1
v11-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v11-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 6ebb1adb6d3a8b9c314a20d9ebee5636ccdc73be Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 19 Nov 2020 13:06:16 +1100
Subject: [PATCH v11 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1063 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 526 ++++++++++++
5 files changed, 1594 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..938d51a 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..5f0a89f
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1063 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index ae89ed7..4fa4b97 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -88,6 +88,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 525bdc8..261cab7 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..9447120
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,526 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
1.8.3.1
Hi
Posting an updated set of patches to address recent feedback:
- Removed conditional-locking code used in parallel-safety checking code
(Tsunakawa-san feedback). It turns out that for the problem test case, no
parallel-safety checking should be occurring that locks relations because
those inserts are specifying VALUES, not an underlying SELECT, so the
parallel-safety checking code was updated to bail out early if no underlying
SELECT is specified for the INSERT. No change to the test code was required.
- Added comment to better explain the reason for treating "INSERT ...
ON CONFLICT ... DO UPDATE" as parallel-unsafe (Dilip)
- Added assertion to heap_prepare_insert() (Amit)
- Updated error handling for NULL index_expr_item case (Vignesh)
+
+ index_oid_list = RelationGetIndexList(rel);
...
As memtioned in the comments of RelationGetIndexList:
* we return a copy of the list palloc'd in the caller's context. The caller
* may list_free() the returned list after scanning it.
Shall we list_free(index_oid_list) at the end of function ?
Just to avoid potential memory leak.
Best regards,
houzj
Hi
I have an issue about the parallel-safety checks.
If target table is foreign table or temporary table,
rel_max_parallel_hazard_for_modify will return PROPARALLEL_UNSAFE,
which not only disable parallel insert but also disable underlying parallel SELECT.
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
I may be wrong, and if I miss sth in previous mails, please give me some hints.
IMO, serial insertion with underlying parallel SELECT can be considered for foreign table or temporary table,
as the insertions only happened in the leader process.
Are there any special considerations for this case ?
Best regards,
houzj
On Wed, Dec 23, 2020 at 7:15 AM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
I have an issue about the parallel-safety checks.
If target table is foreign table or temporary table,
rel_max_parallel_hazard_for_modify will return PROPARALLEL_UNSAFE,
which not only disable parallel insert but also disable underlying parallel SELECT.+create temporary table temp_names (like names); +explain (costs off) insert into temp_names select * from names; + QUERY PLAN +------------------------- + Insert on temp_names + -> Seq Scan on names +(2 rows)I may be wrong, and if I miss sth in previous mails, please give me some hints.
IMO, serial insertion with underlying parallel SELECT can be considered for foreign table or temporary table,
as the insertions only happened in the leader process.
I don't think we support parallel scan for temporary tables. Can you
please try once both of these operations without Insert being
involved? If you are able to produce a parallel plan without Insert
then we can see why it is not supported with Insert.
--
With Regards,
Amit Kapila.
Hi
I may be wrong, and if I miss sth in previous mails, please give me some
hints.
IMO, serial insertion with underlying parallel SELECT can be
considered for foreign table or temporary table, as the insertions onlyhappened in the leader process.
I don't think we support parallel scan for temporary tables. Can you please
try once both of these operations without Insert being involved? If you
are able to produce a parallel plan without Insert then we can see why it
is not supported with Insert.
Sorry, may be I did not express it clearly, I actually means the case when insert's target(not in select part) table is temporary.
And you are right that parallel select is not enabled when temporary table is in select part.
I test for the case when insert's target table is temporary or not.
--insert into not temporary table---
postgres=# explain (costs off) insert into notemp select * from test where i < 600;
QUERY PLAN
---------------------------------------
Gather
Workers Planned: 4
-> Insert on notemp
-> Parallel Seq Scan on test
Filter: (i < 600)
--insert into temporary table---
postgres=# explain (costs off) insert into temp select * from test where i < 600;
QUERY PLAN
---------------------------
Insert on temp
-> Seq Scan on test
Filter: (i < 600)
---without insert part---
postgres=# explain (costs off) select * from test where i < 600;
QUERY PLAN
---------------------------------
Gather
Workers Planned: 4
-> Parallel Seq Scan on test
Filter: (i < 600)
Best regards,
houzj
On Wed, Dec 23, 2020 at 7:52 AM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
I may be wrong, and if I miss sth in previous mails, please give me some
hints.
IMO, serial insertion with underlying parallel SELECT can be
considered for foreign table or temporary table, as the insertions onlyhappened in the leader process.
I don't think we support parallel scan for temporary tables. Can you please
try once both of these operations without Insert being involved? If you
are able to produce a parallel plan without Insert then we can see why it
is not supported with Insert.Sorry, may be I did not express it clearly, I actually means the case when insert's target(not in select part) table is temporary.
And you are right that parallel select is not enabled when temporary table is in select part.
I think Select can be parallel for this case and we should support this case.
--
With Regards,
Amit Kapila.
Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
Following is my review.
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch
--------------------------------------------------------------
@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
Is it really ok to allow PROPARALLEL_RESTRICTED? Per definition, these
functions should not be called by parallel worker.
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
Isn't the test of force_parallel_mode just a hack to make regression tests
pass? When I removed this part and ran the regression tests with
force_parallel_mode=regress, the assertion fired when executing a subquery
because the executor was already in parallel mode due to the main query
execution. I think the function should be implemented such that it does not
mind repeated execution by the same backend.
As an alternative, have you considered allocation of the XID even in parallel
mode? I imagine that the first parallel worker that needs the XID for
insertions allocates it and shares it with the other workers as well as with
the leader process.
One problem of the current patch version is that the "INSERT INTO ... SELECT
..." statement consumes XID even if the SELECT eventually does not return any
row. However, if the same query is processed w/o parallelism, the XID is only
allocated if at least one tuple needs to be inserted.
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch
-------------------------------------------------------------------
@@ -1021,12 +1039,15 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);
if (IsModifySupportedInParallelMode(commandType))
{
+ if (isParallelModifyLeader)
+ (void) GetCurrentCommandId(true);
I miss a comment here. I suppose this is to set currentCommandIdUsed, so that
the leader process gets a new commandId for the following statements in the
same transaction, and thus it can see the rows inserted by the parallel
workers?
If my understanding is correct, I think that the leader should not participate
in the execution of the Insert node, else it would use higher commandId than
the workers. That would be weird, although probably not data corruption. I
wonder if parallel_leader_participation should be considered false for the
"Gather -> Insert -> ..." plans.
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isParallelModifyLeader = false;
+ bool isParallelModifyWithReturning = false;
The variable names are quite long. Since this code deals with the Gather node,
I think that both "Parallel" and "Leader" components can be removed.
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isParallelModifyLeader;
Likewise, the variable name.
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
Is this change needed? The code just before this test indicates that nreaders
should be equal to nworkers_launched.
In grouping_planner(), this branch
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
is very similar to creation of the non-parallel ModifyTablePaths - perhaps an
opportunity to move the common code into a new function.
@@ -2401,6 +2494,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_count > 0)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
A boolean variable (e.g. have_parallel_modify_paths) would suffice, there's no
need to count the paths using parallel_modify_partial_path_count.
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
I'm not sure if the problem of missing targetlist should be handled here (BTW,
NIL is the constant for an empty list, not NULL). Obviously this is a
consequence of the fact that the ModifyTable node has no regular targetlist.
Actually I don't quite understand why (in the current master branch) the
targetlist initialized in set_plan_refs()
/*
* Set up the visible plan targetlist as being the same as
* the first RETURNING list. This is for the use of
* EXPLAIN; the executor won't pay any attention to the
* targetlist. We postpone this step until here so that
* we don't have to do set_returning_clause_references()
* twice on identical targetlists.
*/
splan->plan.targetlist = copyObject(linitial(newRL));
is not used. Instead, ExecInitModifyTable() picks the first returning list
again:
/*
* Initialize result tuple slot and assign its rowtype using the first
* RETURNING list. We assume the rest will look the same.
*/
mtstate->ps.plan->targetlist = (List *) linitial(node->returningLists);
So if you set the targetlist in create_modifytable_plan() (according to
best_path->returningLists), or even in create_modifytable_path(), and ensure
that it gets propagated to the Gather node (generate_gather_pahs currently
uses rel->reltarget), then you should no longer need to tweak
setrefs.c. Moreover, ExecInitModifyTable() would no longer need to set the
targetlist. However I don't guarantee that this is the best approach - some
planner expert should speak up.
--
Antonin Houska
Web: https://www.cybertec-postgresql.com
On Wed, Jan 6, 2021 at 2:09 PM Antonin Houska <ah@cybertec.at> wrote:
Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
Following is my review.
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch
--------------------------------------------------------------@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions, /* all the cheap tests pass, so scan the query tree */ glob->maxParallelHazard = max_parallel_hazard(parse); glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + + /* + * Additional parallel-mode safety checks are required in order to + * allow an underlying parallel query to be used for a + * table-modification command that is supported in parallel-mode. + */ + if (glob->parallelModeOK && + IsModifySupportedInParallelMode(parse->commandType)) + { + glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard); + glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + }Is it really ok to allow PROPARALLEL_RESTRICTED? Per definition, these
functions should not be called by parallel worker.
What in the above change indicates that the parallel_restricted will
be allowed in parallel workers. This just sets paralleModeOK to allow
parallel plans for Selects if the Insert can be performed safely in a
leader backend.
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}/* + * PrepareParallelMode + * + * Prepare for entering parallel mode, based on command-type. + */ +void +PrepareParallelMode(CmdType commandType) +{ + Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);Isn't the test of force_parallel_mode just a hack to make regression tests
pass? When I removed this part and ran the regression tests with
force_parallel_mode=regress, the assertion fired when executing a subquery
because the executor was already in parallel mode due to the main query
execution.
I think this Assert is bogus. We are allowed to enter in parallel-mode
if we are already in parallel-mode, see EnterParallelMode. But we
shouldn't be allowed allocate xid in parallel-mode. So the
Assert(!IsInParallelMode()) should be moved inside the check if
(IsModifySupportedInParallelMode(commandType)) in this function. Can
you check if it still fails after such a modification?
As an alternative, have you considered allocation of the XID even in parallel
mode? I imagine that the first parallel worker that needs the XID for
insertions allocates it and shares it with the other workers as well as with
the leader process.
As a matter of this patch
(v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch), we
never need to allocate xids by workers because Insert is always
performed by leader backend. Even, if we want to do what you are
suggesting it would be tricky because currently, we don't have such an
infrastructure where we can pass information among workers.
One problem of the current patch version is that the "INSERT INTO ... SELECT
..." statement consumes XID even if the SELECT eventually does not return any
row. However, if the same query is processed w/o parallelism, the XID is only
allocated if at least one tuple needs to be inserted.
Yeah, that is true but I think this can happen w/o parallelism for
updates and deletes where by the time we try to modify the row, it got
modified by a concurrent session and the first session will needlessly
allocate XID.
--
With Regards,
Amit Kapila.
Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 6, 2021 at 2:09 PM Antonin Houska <ah@cybertec.at> wrote:
Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
Following is my review.
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch
--------------------------------------------------------------@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions, /* all the cheap tests pass, so scan the query tree */ glob->maxParallelHazard = max_parallel_hazard(parse); glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + + /* + * Additional parallel-mode safety checks are required in order to + * allow an underlying parallel query to be used for a + * table-modification command that is supported in parallel-mode. + */ + if (glob->parallelModeOK && + IsModifySupportedInParallelMode(parse->commandType)) + { + glob->maxParallelHazard = max_parallel_hazard_for_modify(parse, &glob->maxParallelHazard); + glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); + }Is it really ok to allow PROPARALLEL_RESTRICTED? Per definition, these
functions should not be called by parallel worker.What in the above change indicates that the parallel_restricted will
be allowed in parallel workers. This just sets paralleModeOK to allow
parallel plans for Selects if the Insert can be performed safely in a
leader backend.
Well, this is just the initial setting, while the distinction between "Gather
-> Insert -> ..." and "Insert -> Gather -> ..." is made later. So I withdraw
my objection.
@@ -1015,6 +1016,27 @@ IsInParallelMode(void)
}/* + * PrepareParallelMode + * + * Prepare for entering parallel mode, based on command-type. + */ +void +PrepareParallelMode(CmdType commandType) +{ + Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);Isn't the test of force_parallel_mode just a hack to make regression tests
pass? When I removed this part and ran the regression tests with
force_parallel_mode=regress, the assertion fired when executing a subquery
because the executor was already in parallel mode due to the main query
execution.I think this Assert is bogus. We are allowed to enter in parallel-mode
if we are already in parallel-mode, see EnterParallelMode.
Right.
But we shouldn't be allowed allocate xid in parallel-mode. So the
Assert(!IsInParallelMode()) should be moved inside the check if
(IsModifySupportedInParallelMode(commandType)) in this function. Can you
check if it still fails after such a modification?
Yes, this works.
As an alternative, have you considered allocation of the XID even in parallel
mode? I imagine that the first parallel worker that needs the XID for
insertions allocates it and shares it with the other workers as well as with
the leader process.As a matter of this patch
(v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch), we
never need to allocate xids by workers because Insert is always
performed by leader backend.
When writing this comment, I was actually thinking of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch rather
than v11-0001, see below. On the other hand, if we allowed XID allocation in
the parallel mode (as a separate patch), even the 0001 patch would get a bit
simpler.
Even, if we want to do what you are suggesting it would be tricky because
currently, we don't have such an infrastructure where we can pass
information among workers.
How about barriers (storage/ipc/barrier.c)? What I imagine is that all the
workers "meet" at the barrier when they want to insert the first tuple. Then
one of them allocates the XID, makes it available to others (via shared
memory) and all the workers can continue.
One problem of the current patch version is that the "INSERT INTO ... SELECT
..." statement consumes XID even if the SELECT eventually does not return any
row. However, if the same query is processed w/o parallelism, the XID is only
allocated if at least one tuple needs to be inserted.
Yeah, that is true but I think this can happen w/o parallelism for
updates and deletes where by the time we try to modify the row, it got
modified by a concurrent session and the first session will needlessly
allocate XID.
--
Antonin Houska
Web: https://www.cybertec-postgresql.com
On Wed, Jan 6, 2021 at 2:09 PM Antonin Houska <ah@cybertec.at> wrote:
Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
Following is my review.
..
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch
-------------------------------------------------------------------@@ -1021,12 +1039,15 @@ IsInParallelMode(void) * Prepare for entering parallel mode, based on command-type. */ void -PrepareParallelMode(CmdType commandType) +PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader) { Assert(!IsInParallelMode() || force_parallel_mode != FORCE_PARALLEL_OFF);if (IsModifySupportedInParallelMode(commandType)) { + if (isParallelModifyLeader) + (void) GetCurrentCommandId(true);I miss a comment here. I suppose this is to set currentCommandIdUsed, so that
the leader process gets a new commandId for the following statements in the
same transaction, and thus it can see the rows inserted by the parallel
workers?
oh no, leader backend and worker backends must use the same commandId.
I am also not sure if we need this because for Insert statements we
already call GetCurrentCommandId(true) is standard_ExecutorStart. We
don't want the rows visibility behavior for parallel-inserts any
different than non-parallel ones.
If my understanding is correct, I think that the leader should not participate
in the execution of the Insert node, else it would use higher commandId than
the workers. That would be weird, although probably not data corruption.
Yeah, exactly this is the reason both leader and backends must use the
same commandId.
I
wonder if parallel_leader_participation should be considered false for the
"Gather -> Insert -> ..." plans.
If what I said above is correct then this is moot.
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}/* Run plan locally if no workers or enabled and not single-copy. */ - node->need_to_scan_locally = (node->nreaders == 0) + node->need_to_scan_locally = (node->nworkers_launched <= 0) || (!gather->single_copy && parallel_leader_participation); node->initialized = true; }Is this change needed? The code just before this test indicates that nreaders
should be equal to nworkers_launched.
This change is required because we don't need to set up readers for
parallel-insert unless there is a returning clause. See the below
check a few lines before this change:
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader &&
!isParallelModifyWithReturning))
{
I think this check could be simplified to if (pcxt->nworkers_launched
0 && isParallelModifyWithReturning) or something like that.
In grouping_planner(), this branch
+ /* Consider a supported parallel table-modification command */ + if (IsModifySupportedInParallelMode(parse->commandType) && + !inheritance_update && + final_rel->consider_parallel && + parse->rowMarks == NIL) + {is very similar to creation of the non-parallel ModifyTablePaths - perhaps an
opportunity to move the common code into a new function.
+1.
@@ -2401,6 +2494,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}+ if (parallel_modify_partial_path_count > 0) + { + final_rel->rows = current_rel->rows; /* ??? why hasn't this been + * set above somewhere ???? */ + generate_useful_gather_paths(root, final_rel, false); + } + extra.limit_needed = limit_needed(parse); extra.limit_tuples = limit_tuples; extra.count_est = count_est;A boolean variable (e.g. have_parallel_modify_paths) would suffice, there's no
need to count the paths using parallel_modify_partial_path_count.
Sounds sensible.
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}/* Now fix the Plan tree */ - return set_plan_refs(root, plan, rtoffset); + finalPlan = set_plan_refs(root, plan, rtoffset); + if (finalPlan != NULL && IsA(finalPlan, Gather)) + { + Plan *subplan = outerPlan(finalPlan); + + if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL) + { + finalPlan->targetlist = copyObject(subplan->targetlist); + } + } + return finalPlan; }I'm not sure if the problem of missing targetlist should be handled here (BTW,
NIL is the constant for an empty list, not NULL). Obviously this is a
consequence of the fact that the ModifyTable node has no regular targetlist.
I think it is better to add comments along with this change. In this
form, this looks quite hacky to me.
Actually I don't quite understand why (in the current master branch) the
targetlist initialized in set_plan_refs()/*
* Set up the visible plan targetlist as being the same as
* the first RETURNING list. This is for the use of
* EXPLAIN; the executor won't pay any attention to the
* targetlist. We postpone this step until here so that
* we don't have to do set_returning_clause_references()
* twice on identical targetlists.
*/
splan->plan.targetlist = copyObject(linitial(newRL));is not used. Instead, ExecInitModifyTable() picks the first returning list
again:/*
* Initialize result tuple slot and assign its rowtype using the first
* RETURNING list. We assume the rest will look the same.
*/
mtstate->ps.plan->targetlist = (List *) linitial(node->returningLists);So if you set the targetlist in create_modifytable_plan() (according to
best_path->returningLists), or even in create_modifytable_path(), and ensure
that it gets propagated to the Gather node (generate_gather_pahs currently
uses rel->reltarget), then you should no longer need to tweak
setrefs.c.
This sounds worth investigating.
Moreover, ExecInitModifyTable() would no longer need to set the
targetlist.
I am not sure if we need to do anything about ExecInitModifyTable. If
we want to unify what setrefs.c does with ExecInitModifyTable, then we
can start a separate thread.
Thanks for all the reviews. I would like to emphasize what I said
earlier in this thread that it is better to first focus on
Parallelising Selects for Insert (aka what
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT does) as that
in itself is a step towards achieving parallel inserts, doing both
0001 and 0003 at the same time can take much more time as both touches
quite intricate parts of the code.
--
With Regards,
Amit Kapila.
On Fri, Dec 11, 2020 at 4:30 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
- Removed conditional-locking code used in parallel-safety checking
code (Tsunakawa-san feedback). It turns out that for the problem test
case, no parallel-safety checking should be occurring that locks
relations because those inserts are specifying VALUES, not an
underlying SELECT, so the parallel-safety checking code was updated to
bail out early if no underlying SELECT is specified for the INSERT. No
change to the test code was required.
- Added comment to better explain the reason for treating "INSERT ...
ON CONFLICT ... DO UPDATE" as parallel-unsafe (Dilip)
- Added assertion to heap_prepare_insert() (Amit)
- Updated error handling for NULL index_expr_item case (Vignesh)
Thanks Greg for fixing and posting a new patch.
Few comments:
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1,
stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
For one of the test you can validate that the same transaction has
been used by all the parallel workers, you could use something like
below to validate:
SELECT COUNT(*) FROM (SELECT DISTINCT cmin,xmin FROM para_insert_p1) as dt;
Few includes are not required:
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
This include is not required in nodeModifyTable.c
+#include "catalog/index.h"
+#include "catalog/indexing.h"
@@ -43,7 +49,11 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
+#include "rewrite/rewriteHandler.h"
#include "rewrite/rewriteManip.h"
+#include "storage/lmgr.h"
#include "tcop/tcopprot.h"
The includes indexing.h, rewriteHandler.h & lmgr.h is not required in clauses.c
There are few typos:
+ table and populate it can use a parallel plan. Another
exeption is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a
parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
exeption should be exception
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems resonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelModifyWorkers = path->parallel_workers;
resonable should be reasonable
Regards,
Vignesh
EnterpriseDB: http://www.enterprisedb.com
On Fri, Jan 8, 2021 at 12:21 PM Antonin Houska <ah@cybertec.at> wrote:
Amit Kapila <amit.kapila16@gmail.com> wrote:
As an alternative, have you considered allocation of the XID even in parallel
mode? I imagine that the first parallel worker that needs the XID for
insertions allocates it and shares it with the other workers as well as with
the leader process.As a matter of this patch
(v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch), we
never need to allocate xids by workers because Insert is always
performed by leader backend.When writing this comment, I was actually thinking of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch rather
than v11-0001, see below. On the other hand, if we allowed XID allocation in
the parallel mode (as a separate patch), even the 0001 patch would get a bit
simpler.Even, if we want to do what you are suggesting it would be tricky because
currently, we don't have such an infrastructure where we can pass
information among workers.How about barriers (storage/ipc/barrier.c)? What I imagine is that all the
workers "meet" at the barrier when they want to insert the first tuple. Then
one of them allocates the XID, makes it available to others (via shared
memory) and all the workers can continue.
Even if want to do this I am not sure if we need barriers because
there is no intrinsic need for all workers to stop before allocating
XID. After allocation of XID, we just need some way for other workers
to use it, maybe something like all workers currently synchronizes for
getting the block number to process in parallel sequence scans. But
the question is it really worth because in many cases it would be
already allocated by the time parallel DML operation is started and we
do share it in the beginning? I think if we really want to allow
allocation of xid in parallel-mode then we should also think to allow
it for subtransactions xid not only for main transactions and that
will open up some other use cases. I feel it is better to tackle that
problem separately.
--
With Regards,
Amit Kapila.
Hi Greg, Amit
Cc:hackers
4. Have you checked the overhead of this on the planner for
different kinds of statements like inserts into tables having 100
or 500 partitions? Similarly, it is good to check the overhead of
domain related checks added in the patch.Checking that now and will post results soon.
I am seeing a fair bit of overhead in the planning for the INSERT
parallel-safety checks (mind you, compared to the overall performance
gain, it's not too bad).
Considering the 'real-world' use cases and extreme cases I can imagine, I took 3 kinds of measurements on partition table for the latest patch(V11).
The measurement is mainly focus on small rows because this could be easier to evaluate check overhead among the parallelism optimization.
From current results, the overhead looks acceptable compared to the benefits as Greg said.
Test 1: overhead of parallel insert into thousands partitions and 1 rows per partition.
%reg=(patched-master)/master
all time= Execution Time+ Planning Time
| patched | master | %reg |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
partitions |Execution Time(ms)| Planning Time(ms) | Execution Time(ms) | Planning Time(ms) | %reg(Excution Time) | %reg(all time) |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
1000 | 2281.291 | 25.983 | 9752.145 | 0.208 | -77% | -76% |
2000 | 2303.229 | 50.427 | 9446.221 | 0.227 | -76% | -75% |
4000 | 2303.207 | 100.946 | 9948.743 | 0.211 | -77% | -76% |
6000 | 2411.877 | 152.212 | 9953.114 | 0.210 | -76% | -74% |
10000 | 2467.235 | 260.751 | 10917.494 | 0.284 | -77% | -75% |
Test 2: overhead of parallel insert into thousands partitions and 100 rows per partition.
| patched | master | %reg |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
partitions |Execution Time(ms)| Planning Time(ms) | Execution Time(ms) | Planning Time(ms) | %reg(Excution Time) | %reg(all time) |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
1000 | 2366.620 | 25.787 | 14052.748 | 0.238 | -83% | -83% |
2000 | 2325.171 | 48.780 | 10099.203 | 0.211 | -77% | -76% |
4000 | 2599.344 | 110.978 | 10678.065 | 0.216 | -76% | -75% |
6000 | 2764.070 | 152.929 | 10880.948 | 0.238 | -75% | -73% |
10000 | 3043.658 | 265.297 | 11607.202 | 0.207 | -74% | -71% |
Test 3: overhead of parallel insert into varying number of partitions and inserted rows.
| patched | master | %reg |
-----------|-----------------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
partitions |total table rows |Execution Time(ms)| Planning Time(ms) | Execution Time(ms) | Planning Time(ms) | %reg(Excution Time) | %reg(all time) |
-----------|-----------------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
100 | 10000000 | 11202.021 | 1.593 | 25668.560 | 0.212 | -56% | -56% |
500 | 10000000 | 10290.368 | 12.722 | 25730.860 | 0.214 | -60% | -60% |
1000 | 10000000 | 8946.627 | 24.851 | 26271.026 | 0.219 | -66% | -66% |
2000 | 10000000 | 10615.643 | 50.111 | 25512.692 | 0.231 | -58% | -58% |
4000 | 10000000 | 9056.334 | 105.644 | 26643.383 | 0.217 | -66% | -66% |
------------------------------------------------------------------------------------------------------------------------------------------------------
100 | 1000000 | 2757.670 | 1.493 | 11136.357 | 0.208 | -75% | -75% |
500 | 1000000 | 2810.980 | 12.696 | 11483.715 | 0.206 | -76% | -75% |
1000 | 1000000 | 2773.342 | 24.746 | 13441.169 | 0.214 | -79% | -79% |
2000 | 1000000 | 2856.915 | 51.737 | 10996.621 | 0.226 | -74% | -74% |
4000 | 1000000 | 2942.478 | 100.235 | 11422.699 | 0.220 | -74% | -73% |
------------------------------------------------------------------------------------------------------------------------------------------------------
100 | 100000 | 2257.134 | 1.682 | 9351.511 | 0.226 | -76% | -76% |
500 | 100000 | 2197.570 | 12.452 | 9636.659 | 0.203 | -77% | -77% |
1000 | 100000 | 2188.356 | 24.553 | 9647.583 | 0.202 | -77% | -77% |
2000 | 100000 | 2293.287 | 49.167 | 9365.449 | 0.224 | -76% | -75% |
4000 | 100000 | 2375.935 | 104.562 | 10125.190 | 0.219 | -77% | -76% |
------------------------------------------------------------------------------------------------------------------------------------------------------
100 | 10000 | 2142.086 | 1.506 | 9500.491 | 0.206 | -77% | -77% |
500 | 10000 | 2147.779 | 12.260 | 11746.766 | 0.202 | -82% | -82% |
1000 | 10000 | 2153.286 | 23.900 | 9298.452 | 0.212 | -77% | -77% |
2000 | 10000 | 2303.170 | 52.844 | 9772.971 | 0.217 | -76% | -76% |
However, just like Amit and other hackers concerned, if we want to leave the overhead as it is, we should cover real use case as much as possible in case we find the overhead can't be ignored(then we should consider to reduce the overhead).
So if anyone has some reality use cases(which I didn't include in my results above) need to test on this patch. Please share the info with me, I'd like to do more tests on it.
Regards,
Tang
On Fri, Dec 11, 2020 at 4:30 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches to address recent feedback:
Here is an additional review of
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT. There are
quite a few comments raised on the V11-0001* patch. I suggest first
post a revised version of V11-0001* patch addressing those comments
and then you can separately post a revised version of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.
Few comments:
==============
1.
+char
+max_parallel_hazard_for_modify(Query *parse, const char
*initial_max_parallel_hazard)
{
..
+ return (rel_max_parallel_hazard_for_modify(rte->relid,
parse->commandType, &context, NoLock));
..
}
rel_max_parallel_hazard_for_modify()
{
..
+ rel = table_open(relid, lockmode);
..
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
..
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
}
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?
2.
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ max_hazard = index_expr_max_parallel_hazard_for_modify(rel, context);
+ if (max_parallel_hazard_test(max_hazard, context))
+ {
+ table_close(rel, lockmode);
+ return context->max_hazard;
+ }
Here and at all other similar places, the call to
max_parallel_hazard_test seems redundant because
index_expr_max_parallel_hazard_for_modify would have already done
that. Why can't we just return true/false from
index_expr_max_parallel_hazard_for_modify? The context would have been
already updated for max_hazard.
3.
@@ -342,6 +343,18 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+ if (glob->parallelModeOK &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ glob->maxParallelHazard = max_parallel_hazard_for_modify(parse,
&glob->maxParallelHazard);
+ glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
+ }
}
I don't like this way of checking parallel_hazard for modify. This not
only duplicates some code in max_parallel_hazard_for_modify from
max_parallel_hazard but also appears quite awkward. Can we move
max_parallel_hazard_for_modify inside max_parallel_hazard? Basically,
after calling max_parallel_hazard_walker, we can check for modify
statement and call the new function.
4.
domain_max_parallel_hazard_for_modify()
{
..
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ break;
+ }
..
}
index_expr_max_parallel_hazard_for_modify()
{
..
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ index_close(index_rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return context->max_hazard;
+ }
..
}
It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?
--
With Regards,
Amit Kapila.
On Thu, Jan 14, 2021 at 2:37 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:
Hi Greg, Amit
Cc:hackers4. Have you checked the overhead of this on the planner for
different kinds of statements like inserts into tables having 100
or 500 partitions? Similarly, it is good to check the overhead of
domain related checks added in the patch.Checking that now and will post results soon.
I am seeing a fair bit of overhead in the planning for the INSERT
parallel-safety checks (mind you, compared to the overall performance
gain, it's not too bad).Considering the 'real-world' use cases and extreme cases I can imagine, I took 3 kinds of measurements on partition table for the latest patch(V11).
The measurement is mainly focus on small rows because this could be easier to evaluate check overhead among the parallelism optimization.
From current results, the overhead looks acceptable compared to the benefits as Greg said.
Can we test cases when we have few rows in the Select table (say 1000)
and there 500 or 1000 partitions. In that case, we won't select
parallelism but we have to pay the price of checking parallel-safety
of all partitions. Can you check this with 100, 200, 500, 1000
partitions table?
--
With Regards,
Amit Kapila.
Hello Tang-san,
From: Amit Kapila <amit.kapila16@gmail.com>
Can we test cases when we have few rows in the Select table (say 1000)
and there 500 or 1000 partitions. In that case, we won't select
parallelism but we have to pay the price of checking parallel-safety
of all partitions. Can you check this with 100, 200, 500, 1000
partitions table?
I also wanted to see such an extreme(?) case. The 1,000 rows is not the count per partition but the total count of all partitions.e.g., when # of partitions is 100, # of rows per partition is 10.
Regards
Takayuki Tsunakawa
On Fri, Jan 8, 2021 at 8:25 PM vignesh C <vignesh21@gmail.com> wrote:
For one of the test you can validate that the same transaction has
been used by all the parallel workers, you could use something like
below to validate:
SELECT COUNT(*) FROM (SELECT DISTINCT cmin,xmin FROM para_insert_p1) as dt;
Few includes are not required:
#include "executor/nodeGather.h"This include is not required in nodeModifyTable.c
The includes indexing.h, rewriteHandler.h & lmgr.h is not required in clauses.c
exeption should be exception
resonable should be reasonable
Thanks Vignesh,
I'll be sure to make those updates in the next version of the patches.
Regards,
Greg
On Fri, Jan 15, 2021 at 7:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Here is an additional review of
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT. There are
quite a few comments raised on the V11-0001* patch. I suggest first
post a revised version of V11-0001* patch addressing those comments
and then you can separately post a revised version of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.
1)
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?
Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.
(This is not the case for a partition, in which case the patch code
uses AccessShareLock, as you will see).
And BTW, with asserts enabled, an attempt to table_open() with NoLock
when you haven't already locked the table will fire an assert - see
following code in relation_open():
/*
* If we didn't get the lock ourselves, assert that caller holds one,
* except in bootstrap mode where no locks are used.
*/
Assert(lockmode != NoLock ||
IsBootstrapProcessingMode() ||
CheckRelationLockedByMe(r, AccessShareLock, true));
2)
+ /* + * If there are any index expressions, check that they are parallel-mode + * safe. + */ + max_hazard = index_expr_max_parallel_hazard_for_modify(rel, context); + if (max_parallel_hazard_test(max_hazard, context)) + { + table_close(rel, lockmode); + return context->max_hazard; + }
Here and at all other similar places, the call to
max_parallel_hazard_test seems redundant because
index_expr_max_parallel_hazard_for_modify would have already done
that. Why can't we just return true/false from
index_expr_max_parallel_hazard_for_modify? The context would have been
already updated for max_hazard.
Yes, you're right, it's redundant to call max_parallel_hazard_test(_)
again. The max_hazard can always be retrieved from the context if
needed (rather than explicitly returned), so I'll change this function
(and any similar cases) to just return true if the max_hazard of
interest has been reached.
3)
I don't like this way of checking parallel_hazard for modify. This not
only duplicates some code in max_parallel_hazard_for_modify from
max_parallel_hazard but also appears quite awkward. Can we move
max_parallel_hazard_for_modify inside max_parallel_hazard? Basically,
after calling max_parallel_hazard_walker, we can check for modify
statement and call the new function.
Agree, I'll move it, as you suggest.
4)
domain_max_parallel_hazard_for_modify() { .. + if (isnull) + { + /* + * This shouldn't ever happen, but if it does, log a WARNING + * and return UNSAFE, rather than erroring out. + */ + elog(WARNING, "null conbin for constraint %u", con->oid); + context->max_hazard = PROPARALLEL_UNSAFE; + break; + } .. } index_expr_max_parallel_hazard_for_modify() { .. + if (index_expr_item == NULL) /* shouldn't happen */ + { + index_close(index_rel, lockmode); + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + } .. }
It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?
The checks being done here for "should never happen" cases are THE
SAME as other parts of the Postgres code.
For example, search Postgres code for "null conbin" and you'll find 6
other places in the Postgres code which actually ERROR out if conbin
(binary representation of the constraint) in a pg_constraint tuple is
found to be null.
The cases that you point out in the patch used to also error out in
the same way, but Vignesh suggested changing them to just return
parallel-unsafe instead of erroring-out, which I agree with. Such
cases could surely ever only happen if the DB had been corrupted, so
extremely rare IMHO and most likely to have caused an ERROR elsewhere
before ever reaching here...
I can add some Asserts to the current code, to better alert for such
cases, for when asserts are enabled, but otherwise I think that the
current code is OK (cleaning up other Postgres code is beyond the
scope of the task here).
Regards,
Greg Nancarrow
Fujitsu Australia
From: Amit Kapila <amit.kapila16@gmail.com>
Can we test cases when we have few rows in the Select table (say
1000) and there 500 or 1000 partitions. In that case, we won't
select parallelism but we have to pay the price of checking
parallel-safety of all partitions. Can you check this with 100, 200,
500, 1000 partitions table?I also wanted to see such an extreme(?) case. The 1,000 rows is not
the count per partition but the total count of all partitions.e.g.,
when # of partitions is 100, # of rows per partition is 10.
Below results are in serial plan which select table total rows are 1,000. The Excution Time + Planning Time is still less than unpatched.
(does this patch make some optimizes in serial insert? I'm a little confused here, Because the patched execution time is less than unpatched, but I didn't find information in commit messages about it. If I missed something, please kindly let me know.)
| patched | master | %reg |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
partitions |Execution Time(ms)| Planning Time(ms) | Execution Time(ms) | Planning Time(ms) | %reg(Excution Time) | %reg(all time) |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
100 | 5.294 | 1.581 | 6.951 | 0.037 | -24% | -2% |
200 | 9.666 | 3.068 | 13.681 | 0.043 | -29% | -7% |
500 | 22.742 | 12.061 | 35.928 | 0.125 | -37% | -3% |
1000 | 46.386 | 24.872 | 75.523 | 0.142 | -39% | -6% |
I did another test which made check overhead obvious. this case is not fitting for partition purpose, but I put it here as an extreme case too.
Select table total rows are 1,000, # of partitions is 2000. So only the first 1000 partitions have 1 row per partition, the last 1000 partitions have no data inserted.
| patched | master | %reg |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
partitions |Execution Time(ms)| Planning Time(ms) | Execution Time(ms) | Planning Time(ms) | %reg(Excution Time) | %reg(all time) |
-----------|------------------|--------------------|--------------------|-------------------|---------------------|-----------------|
2000 | 45.758 | 51.697 | 80.272 | 0.136 | -43 | 21% |
Regards,
Tang
From: Tang, Haiying <tanghy.fnst@cn.fujitsu.com>
(does this patch make some optimizes in serial insert? I'm a little confused
here, Because the patched execution time is less than unpatched, but I didn't
find information in commit messages about it. If I missed something, please
kindly let me know.)
I haven't thought of anything yet. Could you show us the output of EXPLAIN (ANALYZE, BUFFERS, VERBOSE) of 1,000 partitions case for the patched and unpatched? If it doesn't show any difference, the output of perf may be necessary next.
(BTW, were all the 1,000 rows stored in the target table?)
I did another test which made check overhead obvious. this case is not fitting
for partition purpose, but I put it here as an extreme case too.
Select table total rows are 1,000, # of partitions is 2000. So only the first 1000
partitions have 1 row per partition, the last 1000 partitions have no data
inserted.
Thank you, that's a good test.
Regards
Takayuki Tsunakawa
On Mon, Jan 18, 2021 at 1:02 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:
From: Amit Kapila <amit.kapila16@gmail.com>
Can we test cases when we have few rows in the Select table (say
1000) and there 500 or 1000 partitions. In that case, we won't
select parallelism but we have to pay the price of checking
parallel-safety of all partitions. Can you check this with 100, 200,
500, 1000 partitions table?I also wanted to see such an extreme(?) case. The 1,000 rows is not
the count per partition but the total count of all partitions.e.g.,
when # of partitions is 100, # of rows per partition is 10.Below results are in serial plan which select table total rows are 1,000. The Excution Time + Planning Time is still less than unpatched.
(does this patch make some optimizes in serial insert? I'm a little confused here, Because the patched execution time is less than unpatched, but I didn't find information in commit messages about it. If I missed something, please kindly let me know.)
I don't think the patch should have any impact on the serial case. I
think you can try to repeat each test 3 times both with and without a
patch and take the median of the three.
--
With Regards,
Amit Kapila.
Hi Tsunakawa-san
From: Tang, Haiying <tanghy.fnst@cn.fujitsu.com>
(does this patch make some optimizes in serial insert? I'm a little
confused here, Because the patched execution time is less than
unpatched, but I didn't find information in commit messages about it.
If I missed something, please kindly let me know.)I haven't thought of anything yet. Could you show us the output of
EXPLAIN (ANALYZE, BUFFERS, VERBOSE) of 1,000 partitions case for the
patched and unpatched? If it doesn't show any difference, the output
of perf may be necessary next.
Execute EXPLAIN on Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_part select * from test_data1;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual time=44.139..44.140 rows=0 loops=1)
Buffers: shared hit=1005 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000 width=8) (actual time=0.007..0.201 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=27011
Planning Time: 24.526 ms
Execution Time: 44.981 ms
Execute EXPLAIN on non-Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_part select * from test_data1;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual time=72.656..72.657 rows=0 loops=1)
Buffers: shared hit=22075 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000 width=8) (actual time=0.010..0.175 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=72
Planning Time: 0.135 ms
Execution Time: 79.058 ms
(BTW, were all the 1,000 rows stored in the target table?)
Yes, I checked all rows stored in target table.
postgres=# select count(*) from test_part; count
-------
1000
Regards,
Tang
On Mon, Jan 18, 2021 at 10:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Jan 15, 2021 at 7:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Here is an additional review of
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT. There are
quite a few comments raised on the V11-0001* patch. I suggest first
post a revised version of V11-0001* patch addressing those comments
and then you can separately post a revised version of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.1)
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.
Fair enough. I suggest adding a comment saying the same so that the
reader doesn't get confused about the same.
(This is not the case for a partition, in which case the patch code
uses AccessShareLock, as you will see).
Okay, but I see you release the lock on partition rel immediately
after checking parallel-safety. What if a user added some
parallel-unsafe constraint (via Alter Table) after that check?
4)
domain_max_parallel_hazard_for_modify() { .. + if (isnull) + { + /* + * This shouldn't ever happen, but if it does, log a WARNING + * and return UNSAFE, rather than erroring out. + */ + elog(WARNING, "null conbin for constraint %u", con->oid); + context->max_hazard = PROPARALLEL_UNSAFE; + break; + } .. } index_expr_max_parallel_hazard_for_modify() { .. + if (index_expr_item == NULL) /* shouldn't happen */ + { + index_close(index_rel, lockmode); + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + } .. }It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?The checks being done here for "should never happen" cases are THE
SAME as other parts of the Postgres code.
For example, search Postgres code for "null conbin" and you'll find 6
other places in the Postgres code which actually ERROR out if conbin
(binary representation of the constraint) in a pg_constraint tuple is
found to be null.
The cases that you point out in the patch used to also error out in
the same way, but Vignesh suggested changing them to just return
parallel-unsafe instead of erroring-out, which I agree with.
You have not raised a WARNING for the second case. But in the first
place what is the reasoning for making this different from other parts
of code? If we don't have a solid reason then I suggest keeping these
checks and errors the same as in other parts of the code.
--
With Regards,
Amit Kapila.
Hi Amit
I don't think the patch should have any impact on the serial case. I
think you can try to repeat each test 3 times both with and without a
patch and take the median of the three.
Actually, I repeated about 10 times, the execution time is always less than unpatched.
Regards,
Tang
On Mon, Jan 18, 2021 at 2:40 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:
Hi Tsunakawa-san
From: Tang, Haiying <tanghy.fnst@cn.fujitsu.com>
(does this patch make some optimizes in serial insert? I'm a little
confused here, Because the patched execution time is less than
unpatched, but I didn't find information in commit messages about it.
If I missed something, please kindly let me know.)I haven't thought of anything yet. Could you show us the output of
EXPLAIN (ANALYZE, BUFFERS, VERBOSE) of 1,000 partitions case for the
patched and unpatched? If it doesn't show any difference, the output
of perf may be necessary next.Execute EXPLAIN on Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_part select * from test_data1;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual time=44.139..44.140 rows=0 loops=1)
Buffers: shared hit=1005 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000 width=8) (actual time=0.007..0.201 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=27011
Planning Time: 24.526 ms
Execution Time: 44.981 msExecute EXPLAIN on non-Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_part select * from test_data1;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual time=72.656..72.657 rows=0 loops=1)
Buffers: shared hit=22075 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000 width=8) (actual time=0.010..0.175 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=72
Planning Time: 0.135 ms
Execution Time: 79.058 ms
So, the results indicate that after the patch we touch more buffers
during planning which I think is because of accessing the partition
information, and during execution, the patch touches fewer buffers for
the same reason. But why this can reduce the time with patch? I think
this needs some investigation.
--
With Regards,
Amit Kapila.
On Mon, Jan 18, 2021 at 2:42 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Jan 18, 2021 at 10:45 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Jan 15, 2021 at 7:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Here is an additional review of
v11-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT. There are
quite a few comments raised on the V11-0001* patch. I suggest first
post a revised version of V11-0001* patch addressing those comments
and then you can separately post a revised version of
v11-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.1)
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.Fair enough. I suggest adding a comment saying the same so that the
reader doesn't get confused about the same.(This is not the case for a partition, in which case the patch code
uses AccessShareLock, as you will see).Okay, but I see you release the lock on partition rel immediately
after checking parallel-safety. What if a user added some
parallel-unsafe constraint (via Alter Table) after that check?4)
domain_max_parallel_hazard_for_modify() { .. + if (isnull) + { + /* + * This shouldn't ever happen, but if it does, log a WARNING + * and return UNSAFE, rather than erroring out. + */ + elog(WARNING, "null conbin for constraint %u", con->oid); + context->max_hazard = PROPARALLEL_UNSAFE; + break; + } .. } index_expr_max_parallel_hazard_for_modify() { .. + if (index_expr_item == NULL) /* shouldn't happen */ + { + index_close(index_rel, lockmode); + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + } .. }It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?The checks being done here for "should never happen" cases are THE
SAME as other parts of the Postgres code.
For example, search Postgres code for "null conbin" and you'll find 6
other places in the Postgres code which actually ERROR out if conbin
(binary representation of the constraint) in a pg_constraint tuple is
found to be null.
The cases that you point out in the patch used to also error out in
the same way, but Vignesh suggested changing them to just return
parallel-unsafe instead of erroring-out, which I agree with.You have not raised a WARNING for the second case. But in the first
place what is the reasoning for making this different from other parts
of code?
On again, thinking about this, I see a reason why one wants to do like
you have done currently in the patch. It helps us to avoid giving such
errors when they are really not required say when it occurred while
checking parallel-safety for a particular partition and in reality we
will never insert in that partition and there probably similar other
cases. I guess we should give WARNING consistently in all such cases.
--
With Regards,
Amit Kapila.
On Mon, Jan 18, 2021 at 8:10 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
1)
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.Fair enough. I suggest adding a comment saying the same so that the
reader doesn't get confused about the same.
OK, I'll add a comment.
(This is not the case for a partition, in which case the patch code
uses AccessShareLock, as you will see).Okay, but I see you release the lock on partition rel immediately
after checking parallel-safety. What if a user added some
parallel-unsafe constraint (via Alter Table) after that check?
I'm not sure. But there would be a similar concern for current
Parallel SELECT functionality, right?
My recollection is that ALTER TABLE obtains an exclusive lock on the
table which it retains until the end of the transaction, so that will
result in blocking at certain points, during parallel-checks and
execution, but there may still be a window.
4)
domain_max_parallel_hazard_for_modify() { .. + if (isnull) + { + /* + * This shouldn't ever happen, but if it does, log a WARNING + * and return UNSAFE, rather than erroring out. + */ + elog(WARNING, "null conbin for constraint %u", con->oid); + context->max_hazard = PROPARALLEL_UNSAFE; + break; + } .. } index_expr_max_parallel_hazard_for_modify() { .. + if (index_expr_item == NULL) /* shouldn't happen */ + { + index_close(index_rel, lockmode); + context->max_hazard = PROPARALLEL_UNSAFE; + return context->max_hazard; + } .. }It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?The checks being done here for "should never happen" cases are THE
SAME as other parts of the Postgres code.
For example, search Postgres code for "null conbin" and you'll find 6
other places in the Postgres code which actually ERROR out if conbin
(binary representation of the constraint) in a pg_constraint tuple is
found to be null.
The cases that you point out in the patch used to also error out in
the same way, but Vignesh suggested changing them to just return
parallel-unsafe instead of erroring-out, which I agree with.You have not raised a WARNING for the second case.
The same checks in current Postgres code also don't raise a WARNING
for that case, so I'm just being consistent with existing Postgres
code (which itself isn't consistent for those two cases).
But in the first
place what is the reasoning for making this different from other parts
of code? If we don't have a solid reason then I suggest keeping these
checks and errors the same as in other parts of the code.
The checks are the same as done in existing Postgres source - but
instead of failing with an ERROR (i.e. whole backend dies), in the
middle of parallel-safety-checking, it has been changed to regard the
operation as parallel-unsafe, so that it will try to execute in
non-parallel mode, where it will most likely fail too when those
corrupted attributes are accessed - but it will fail in the way that
it currently does in Postgres, should that very rare condition ever
happen. This was suggested by Vignesh, and I agree with him. So in
effect, it's just allowing it to use the existing error paths in the
code, rather than introducing new ERROR points.
Regards,
Greg Nancarrow
Fujitsu Australia
From: Greg Nancarrow <gregn4422@gmail.com>
On Fri, Jan 15, 2021 at 7:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.
Isn't there any case where planning is done but parse analysis is not done immediately before? e.g.
* Alteration of objects invalidates cached query plans, and the next execution of the plan rebuilds it. (But it seems that parse analysis is done in this case in plancache.c.)
* Execute a prepared statement with a different parameter value, which builds a new custom plan or a generic plan.
Is the cached query plan invalidated when some alteration is done to change the parallel safety, such as adding a trigger with a parallel-unsafe trigger action?
Regards
Takayuki Tsunakawa
On Mon, Jan 18, 2021 at 3:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Jan 18, 2021 at 8:10 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
It is not clear why the above two are shouldn't happen cases and if so
why we want to treat them as unsafe. Ideally, there should be an
Assert if these can't happen but it is difficult to decide without
knowing why you have considered them unsafe?The checks being done here for "should never happen" cases are THE
SAME as other parts of the Postgres code.
For example, search Postgres code for "null conbin" and you'll find 6
other places in the Postgres code which actually ERROR out if conbin
(binary representation of the constraint) in a pg_constraint tuple is
found to be null.
The cases that you point out in the patch used to also error out in
the same way, but Vignesh suggested changing them to just return
parallel-unsafe instead of erroring-out, which I agree with.You have not raised a WARNING for the second case.
The same checks in current Postgres code also don't raise a WARNING
for that case, so I'm just being consistent with existing Postgres
code (which itself isn't consistent for those two cases).
Search for the string "too few entries in indexprs list" and you will
find a lot of places in code raising ERROR for the same condition.
--
With Regards,
Amit Kapila.
On Tue, Jan 19, 2021 at 2:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
You have not raised a WARNING for the second case.
The same checks in current Postgres code also don't raise a WARNING
for that case, so I'm just being consistent with existing Postgres
code (which itself isn't consistent for those two cases).Search for the string "too few entries in indexprs list" and you will
find a lot of places in code raising ERROR for the same condition.
Yes, but raising an ERROR stops processing (not just logs an error
message). Raising a WARNING logs a warning message and continues
processing. It's a big difference.
So, for the added parallel-safety-checking code, it was suggested by
Vignesh (and agreed by me) that, for these rare and highly unlikely
conditions, it would be best not to just copy the error-handling code
verbatim from other cases in the Postgres code (as I had originally
done) and just stop processing dead with an error, but to instead
return PARALLEL_UNSAFE, so that processing continues as it would for
current non-parallel processing, which would most likely error-out
anyway along the current error-handling checks and paths when those
bad attributes/fields are referenced.
I will add some Asserts() and don't mind adding a WARNING message for
the 2nd case.
If you really feel strongly about this, I can just restore the
original code, which will stop dead with an ERROR in the middle of
parallel-safety checking should one of these rare conditions ever
occur.
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Jan 19, 2021 at 1:37 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.Isn't there any case where planning is done but parse analysis is not done immediately before? e.g.
* Alteration of objects invalidates cached query plans, and the next execution of the plan rebuilds it. (But it seems that parse analysis is done in this case in plancache.c.)
* Execute a prepared statement with a different parameter value, which builds a new custom plan or a generic plan.
I don't know, but since NoLock is used in other parts of the planner,
I'd expect those to fail if such cases existed.
Is the cached query plan invalidated when some alteration is done to change the parallel safety, such as adding a trigger with a parallel-unsafe trigger action?
Needs to be tested, but I'd expect the cached plan to get invalidated
in this case - surely the same potential issue exists in Postgres for
the current Parallel SELECT functionality - for example, for a column
with a default value that is an expression (which could be altered
from being parallel-safe to parallel-unsafe).
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Jan 18, 2021 at 3:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Jan 18, 2021 at 8:10 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
1)
Here, it seems we are accessing the relation descriptor without any
lock on the table which is dangerous considering the table can be
modified in a parallel session. Is there a reason why you think this
is safe? Did you check anywhere else such a coding pattern?Yes, there's a very good reason and I certainly have checked for the
same coding pattern elsewhere, and not just randomly decided that
locking can be ignored.
The table has ALREADY been locked (by the caller) during the
parse/analyze phase.Fair enough. I suggest adding a comment saying the same so that the
reader doesn't get confused about the same.OK, I'll add a comment.
(This is not the case for a partition, in which case the patch code
uses AccessShareLock, as you will see).Okay, but I see you release the lock on partition rel immediately
after checking parallel-safety. What if a user added some
parallel-unsafe constraint (via Alter Table) after that check?I'm not sure. But there would be a similar concern for current
Parallel SELECT functionality, right?
I don't think so because, for Selects, we take locks on the required
partitions and don't release them immediately. We do parallel safety
checks after acquiring those locks. From the code perspective, we lock
individual partitions via
expand_inherited_rtentry->expand_partitioned_rtentry and then check
parallel-safety at a later point via
set_append_rel_size->set_rel_consider_parallel. Also, I am not sure if
there is anything we check for Selects at each partition relation
level that can be changed by a concurrent session. Do you have a
different understanding?
Similarly, we do retain locks on indexes, see get_relation_info, which
we are not doing in the patch.
My recollection is that ALTER TABLE obtains an exclusive lock on the
table which it retains until the end of the transaction, so that will
result in blocking at certain points, during parallel-checks and
execution, but there may still be a window.
Once the Select has acquired locks in the above code path, I don't
think Alter for a particular partition would be able to proceed unless
those locks are non-conflicting.
--
With Regards,
Amit Kapila.
On Tue, Jan 19, 2021 at 9:19 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Jan 19, 2021 at 2:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
You have not raised a WARNING for the second case.
The same checks in current Postgres code also don't raise a WARNING
for that case, so I'm just being consistent with existing Postgres
code (which itself isn't consistent for those two cases).Search for the string "too few entries in indexprs list" and you will
find a lot of places in code raising ERROR for the same condition.Yes, but raising an ERROR stops processing (not just logs an error
message). Raising a WARNING logs a warning message and continues
processing. It's a big difference.
So, for the added parallel-safety-checking code, it was suggested by
Vignesh (and agreed by me) that, for these rare and highly unlikely
conditions, it would be best not to just copy the error-handling code
verbatim from other cases in the Postgres code (as I had originally
done) and just stop processing dead with an error, but to instead
return PARALLEL_UNSAFE, so that processing continues as it would for
current non-parallel processing, which would most likely error-out
anyway along the current error-handling checks and paths when those
bad attributes/fields are referenced.
I will add some Asserts() and don't mind adding a WARNING message for
the 2nd case.
If you really feel strongly about this, I can just restore the
original code, which will stop dead with an ERROR in the middle of
parallel-safety checking should one of these rare conditions ever
occur.
I am expecting that either we raise a WARNING and return
parallel_unsafe for all such checks (shouldn't reach cases) in the
patch or simply raise an ERROR as we do in other parts of the patch. I
personally prefer the latter alternative but I am fine with the former
one as well.
--
With Regards,
Amit Kapila.
On Tue, Jan 19, 2021 at 10:32 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Jan 19, 2021 at 1:37 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:The table has ALREADY been locked (by the caller) during the
parse/analyze phase.Isn't there any case where planning is done but parse analysis is not done immediately before? e.g.
* Alteration of objects invalidates cached query plans, and the next execution of the plan rebuilds it. (But it seems that parse analysis is done in this case in plancache.c.)
* Execute a prepared statement with a different parameter value, which builds a new custom plan or a generic plan.
I don't know, but since NoLock is used in other parts of the planner,
I'd expect those to fail if such cases existed.
I think I know how for both the above cases, we ensure that the locks
are acquired before we reach the planner. It seems we will call
GetCachedPlan during these scenarios which will acquire the required
locks in RevalidateCachedQuery both when the cached plan is invalid
and when it is valid. So, we should be fine even when the
custom/generic plan needs to be formed due to a different parameter.
Is the cached query plan invalidated when some alteration is done to change the parallel safety, such as adding a trigger with a parallel-unsafe trigger action?
Needs to be tested,
Yeah, it would be good to test it but I think even if the plan is
invalidated, we will reacquire the required locks as mentioned above.
Tsunakawa-San, does this address your concerns around locking the
target relation in the required cases? It would be good to test but I
don't see any problems in the scenarios you mentioned.
--
With Regards,
Amit Kapila.
From: Amit Kapila <amit.kapila16@gmail.com>
Tsunakawa-San, does this address your concerns around locking the
target relation in the required cases? It would be good to test but I
don't see any problems in the scenarios you mentioned.
Thank you, understood. RevalidateCachedQuery() does parse analysis, that's the trick.
Regards
Takayuki Tsunakawa
On Fri, Jan 8, 2021 at 8:25 PM vignesh C <vignesh21@gmail.com> wrote:
Few includes are not required:
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;This include is not required in nodeModifyTable.c
I think you meant nodeGather.c (not nodeModifyTable.c).
However, the include file (executor/nodeModifyTable.h) is actually
required here, otherwise there are build warnings.
Regards,
Greg Nancarrow
Fujitsu Australia
From: Tang, Haiying <tanghy.fnst@cn.fujitsu.com>
Execute EXPLAIN on Patched:
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=44.139..44.140 rows=0 loops=1)
Buffers: shared hit=1005 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000
width=8) (actual time=0.007..0.201 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Execute EXPLAIN on non-Patched:
Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=72.656..72.657 rows=0 loops=1)
Buffers: shared hit=22075 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000
width=8) (actual time=0.010..0.175 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
I don't know if this is related to this issue, but I felt "shared hit=5" for Seq Scan is strange. This test case reads 1,000 rows from 1,000 partitions, one row per partition, so I thought the shared hit should be 1,000 in Seq Scan. I wonder if the 1,000 is included in Insert node?
Regards
Takayuki Tsunakawa
On Fri, Jan 8, 2021 at 8:16 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
- if (pcxt->nworkers_launched > 0) + if (pcxt->nworkers_launched > 0 && !(isParallelModifyLeader && !isParallelModifyWithReturning)) {I think this check could be simplified to if (pcxt->nworkers_launched
0 && isParallelModifyWithReturning) or something like that.
Not quite. The existing check is correct, because it needs to account
for existing Parallel SELECT functionality (not just new Parallel
INSERT).
But I will re-write the test as an equivalent expression, so it's
hopefully more readable (taking into account Antonin's suggested
variable-name changes):
if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}/* Now fix the Plan tree */ - return set_plan_refs(root, plan, rtoffset); + finalPlan = set_plan_refs(root, plan, rtoffset); + if (finalPlan != NULL && IsA(finalPlan, Gather)) + { + Plan *subplan = outerPlan(finalPlan); + + if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL) + { + finalPlan->targetlist = copyObject(subplan->targetlist); + } + } + return finalPlan; }I'm not sure if the problem of missing targetlist should be handled here (BTW,
NIL is the constant for an empty list, not NULL). Obviously this is a
consequence of the fact that the ModifyTable node has no regular targetlist.I think it is better to add comments along with this change. In this
form, this looks quite hacky to me.
The targetlist on the ModifyTable node has been setup correctly, but
it hasn't been propagated to the Gather node.
Of course, I have previously tried to elegantly fix this, but struck
various problems, using different approaches.
Perhaps this code could just be moved into set_plan_refs().
For the moment, I'll just keep the current code, but I'll add a FIXME
comment for this.
I'll investigate Antonin's suggestions as a lower-priority side-task.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Jan 20, 2021 at 3:27 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Jan 8, 2021 at 8:16 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I'm not sure if the problem of missing targetlist should be handled here (BTW,
NIL is the constant for an empty list, not NULL). Obviously this is a
consequence of the fact that the ModifyTable node has no regular targetlist.I think it is better to add comments along with this change. In this
form, this looks quite hacky to me.The targetlist on the ModifyTable node has been setup correctly, but
it hasn't been propagated to the Gather node.
Of course, I have previously tried to elegantly fix this, but struck
various problems, using different approaches.
Perhaps this code could just be moved into set_plan_refs().
For the moment, I'll just keep the current code, but I'll add a FIXME
comment for this.
I'll investigate Antonin's suggestions as a lower-priority side-task.
+1, that sounds like a good approach because anyway this has to be
dealt for the second patch and at this point, we should make the first
patch ready.
--
With Regards,
Amit Kapila.
Thanks for the feedback.
Posting an updated set of patches. Changes are based on feedback, as
detailed below:
There's a couple of potential issues currently being looked at:
- locking issues in additional parallel-safety checks?
- apparent uneven work distribution across the parallel workers, for
large insert data
[Antonin]
- Fixed bad Assert in PrepareParallelMode()
- Added missing comment to explain use of GetCurrentCommandId() in
PrepareParallelMode()
- Some variable name shortening in a few places
- Created common function for creation of non-parallel and parallel
ModifyTable paths
- Path count variable changed to bool
- Added FIXME comment to dubious code for creating Gather target-list
from ModifyTable subplan
- Fixed check on returningLists to use NIL instead of NULL
[Amit]
- Moved additional parallel-safety checks (for modify case) into
max_parallel_hazard()
- Removed redundant calls to max_parallel_hazard_test()
- Added Asserts to "should never happen" null-attribute cases (and
added WARNING log missing from one case)
- Added comment for use of NoLock in max_parallel_hazard_for_modify()
[Vignesh]
- Fixed a couple of typos
- Added a couple of test cases for testing that the same transaction
is used by all parallel workers
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 4028b93c4b1d2fcd2a245b894b6d50b369ddd556 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Tue, 19 Jan 2021 15:12:53 +1100
Subject: [PATCH v12 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 21 +-
src/backend/optimizer/util/clauses.c | 452 +++++++++++++++++++++++++++
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 1 +
6 files changed, 504 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..7514106dc8 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelMode
+ *
+ * Prepare for entering parallel mode, based on command-type.
+ */
+void
+PrepareParallelMode(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index f4dd47acc7..c72e3e102e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelMode(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4e6497ff32..83ca943736 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 51d26a0691..b13d57cb2a 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -148,6 +158,15 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static bool rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -553,6 +572,18 @@ max_parallel_hazard(Query *parse)
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+
+ if (context.max_hazard != PROPARALLEL_UNSAFE &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ context.max_hazard = max_parallel_hazard_for_modify(parse, context.max_hazard);
+ }
return context.max_hazard;
}
@@ -786,6 +817,427 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Oid index_oid = lfirst_oid(lc);
+ Relation index_rel;
+ IndexInfo *index_info;
+
+ index_rel = index_open(index_oid, lockmode);
+
+ index_info = BuildIndexInfo(index_rel);
+
+ if (index_info->ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
+
+ for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ {
+ int keycol = index_info->ii_IndexAttrNumbers[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ index_close(index_rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ index_close(index_rel, lockmode);
+ return true;
+ }
+
+ index_expr_item = lnext(index_info->ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+
+ return false;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+rel_max_parallel_hazard_for_modify(Oid relid,
+ CmdType command_type,
+ max_parallel_hazard_context *context,
+ LOCKMODE lockmode)
+{
+ Relation rel;
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ rel = table_open(relid, lockmode);
+
+ /*
+ * We can't support table modification in parallel-mode if it's a foreign
+ * table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ table_close(rel, lockmode);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (trigger_max_parallel_hazard_for_modify(rel->trigdesc, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
+ }
+ }
+
+ table_close(rel, lockmode);
+ return false;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ ListCell *lc;
+ bool hasSubQuery;
+ max_parallel_hazard_context context;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * If there is no underlying query, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == 0 ||
+ initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == 0 ?
+ PROPARALLEL_SAFE : initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase).
+ */
+ (void) rel_max_parallel_hazard_for_modify(rte->relid, parse->commandType, &context, NoLock);
+ return context.max_hazard;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..5571e8c0a2 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelMode(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index da3fc4df10..cc3cfc16b2 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
2.27.0
v12-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v12-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 529877a574ca2584545b999f43c2bb94cc4dc78c Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 20 Jan 2021 21:52:46 +1100
Subject: [PATCH v12 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1078 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 531 ++++++++
5 files changed, 1614 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..71d63d79b6
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1078 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names5
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names5 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on temp_names
+ -> Seq Scan on names
+(2 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index e0e1ef71dd..c78f213bf8 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 081fce32e7..41ff09791d 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..93b2518f79
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,531 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names4 (like names);
+explain (costs off) insert into names4 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names order by last_name returning *;
+insert into names5 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_expr_t_1 partition of partkey_unsafe_key_expr_t for values from (0) to (5000);
+create table partkey_unsafe_key_expr_t_2 partition of partkey_unsafe_key_expr_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
--
2.27.0
v12-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v12-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From b9414e128ead1b218aa0a60b9653836789c79bc0 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 20 Jan 2021 21:23:26 +1100
Subject: [PATCH v12 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 7 +-
src/backend/executor/execParallel.c | 65 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 19 +-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 504 insertions(+), 164 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index faffbb1865..7e9b38a1d5 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2067,10 +2070,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 7514106dc8..5670e00d6d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c72e3e102e..821900f84d 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..0fcfa96269 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -590,6 +594,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
char *paramlistinfo_space;
BufferUsage *bufusage_space;
WalUsage *walusage_space;
+ uint64 *processed_count_space;
SharedExecutorInstrumentation *instrumentation = NULL;
SharedJitInstrumentation *jit_instrumentation = NULL;
int pstmt_len;
@@ -675,6 +680,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ processed_count_space = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, processed_count_space);
+ pei->processed_count = processed_count_space;
+ }
+ else
+ {
+ pei->processed_count = NULL;
+ }
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1181,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1417,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1507,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 921e695419..009706ce48 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1829,6 +1830,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2155,7 +2189,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2232,7 +2270,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 380336518f..b07e11b7a5 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 25d4750ca6..dad91cfc3b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 83ca943736..92f75f368d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1793,7 +1797,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1841,6 +1958,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2281,96 +2399,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2389,6 +2444,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7558,7 +7620,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..41599ce44e 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,23 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing targetlist
+ * on the Gather node, in the case of an underlying ModifyTable node - currently it's
+ * not clear how to achieve this elegantly.
+ */
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d465b9e213..dd89a63841 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 5571e8c0a2..19bbf60d86 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelMode(CmdType commandType);
+extern void PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 48c3f570fa..45ea51e28a 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1164,7 +1164,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 23dec14cbd..b2ab024e69 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..866bbb82ea 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -575,15 +575,14 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v12-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v12-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 0edbe34d676e74560dc5ff7f79aa5352e2b62530 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 20 Jan 2021 17:00:40 +1100
Subject: [PATCH v12 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 20 +--
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..3cc029c6e3 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 71d63d79b6..acdad62d5e 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -70,14 +70,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -113,7 +113,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -205,9 +205,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -244,9 +244,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -286,9 +286,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -336,9 +336,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -371,9 +371,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -407,9 +407,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -445,9 +445,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -485,9 +485,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -532,21 +532,21 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names4
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names4
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -575,7 +575,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -625,14 +625,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -681,7 +681,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -710,14 +710,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -774,9 +774,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -826,9 +826,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -865,7 +865,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -886,11 +886,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -931,7 +931,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -972,9 +972,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 93b2518f79..6ae2e64a3c 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -89,7 +89,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -101,7 +101,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -234,14 +234,14 @@ select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names4 (like names);
explain (costs off) insert into names4 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names order by last_name returning *;
@@ -249,7 +249,7 @@ insert into names5 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -274,7 +274,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -290,7 +290,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -298,7 +298,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -375,7 +375,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -425,7 +425,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
2.27.0
Thanks for the feedback.
Posting an updated set of patches. Changes are based on feedback, as detailed
below:
Hi
It seems there are some previous comments[1]/messages/by-id/77e1c06ffb2240838e5fc94ec8dcb7d3@G08CNEXMBPEKD05.g08.fujitsu.local[2]/messages/by-id/CAA4eK1LMmz58ej5BgVLJ8VsUGd=+KcaA8X=kStORhxpfpODOxg@mail.gmail.com not addressed in current patch.
Just to make sure it's not missed.
[1]: /messages/by-id/77e1c06ffb2240838e5fc94ec8dcb7d3@G08CNEXMBPEKD05.g08.fujitsu.local
/messages/by-id/77e1c06ffb2240838e5fc94ec8dcb7d3@G08CNEXMBPEKD05.g08.fujitsu.local
[2]: /messages/by-id/CAA4eK1LMmz58ej5BgVLJ8VsUGd=+KcaA8X=kStORhxpfpODOxg@mail.gmail.com
/messages/by-id/CAA4eK1LMmz58ej5BgVLJ8VsUGd=+KcaA8X=kStORhxpfpODOxg@mail.gmail.com
Best regards,
houzj
Hi,
For v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.
existing and 'it currently performs' are redundant. You can omit 'that it
currently performs'.
Minor. For index_expr_max_parallel_hazard_for_modify(),
+ if (keycol == 0)
+ {
+ /* Found an index expression */
You can check if keycol != 0, continue with the loop. This would save some
indent.
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
I think the warning should be an error since there is assertion ahead of
the if statement.
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
The above should be error as well.
Cheers
On Wed, Jan 20, 2021 at 5:06 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Show quoted text
Thanks for the feedback.
Posting an updated set of patches. Changes are based on feedback, as
detailed below:There's a couple of potential issues currently being looked at:
- locking issues in additional parallel-safety checks?
- apparent uneven work distribution across the parallel workers, for
large insert data[Antonin]
- Fixed bad Assert in PrepareParallelMode()
- Added missing comment to explain use of GetCurrentCommandId() in
PrepareParallelMode()
- Some variable name shortening in a few places
- Created common function for creation of non-parallel and parallel
ModifyTable paths
- Path count variable changed to bool
- Added FIXME comment to dubious code for creating Gather target-list
from ModifyTable subplan
- Fixed check on returningLists to use NIL instead of NULL[Amit]
- Moved additional parallel-safety checks (for modify case) into
max_parallel_hazard()
- Removed redundant calls to max_parallel_hazard_test()
- Added Asserts to "should never happen" null-attribute cases (and
added WARNING log missing from one case)
- Added comment for use of NoLock in max_parallel_hazard_for_modify()[Vignesh]
- Fixed a couple of typos
- Added a couple of test cases for testing that the same transaction
is used by all parallel workersRegards,
Greg Nancarrow
Fujitsu Australia
Hi,
For v12-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch:
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
IsA(outerPlanState(planstate), ModifyTableState);
Please wrap long line.
+ uint64 *processed_count_space;
If I read the code correctly, it seems it can be dropped (use
pei->processed_count directly).
Cheers
On Wed, Jan 20, 2021 at 6:29 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Show quoted text
Hi,
For v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.existing and 'it currently performs' are redundant. You can omit 'that it
currently performs'.Minor. For index_expr_max_parallel_hazard_for_modify(),
+ if (keycol == 0) + { + /* Found an index expression */You can check if keycol != 0, continue with the loop. This would save some
indent.+ if (index_expr_item == NULL) /* shouldn't happen */ + { + elog(WARNING, "too few entries in indexprs list");I think the warning should be an error since there is assertion ahead of
the if statement.+ Assert(!isnull); + if (isnull) + { + /* + * This shouldn't ever happen, but if it does, log a WARNING + * and return UNSAFE, rather than erroring out. + */ + elog(WARNING, "null conbin for constraint %u", con->oid);The above should be error as well.
Cheers
On Wed, Jan 20, 2021 at 5:06 PM Greg Nancarrow <gregn4422@gmail.com>
wrote:Thanks for the feedback.
Posting an updated set of patches. Changes are based on feedback, as
detailed below:There's a couple of potential issues currently being looked at:
- locking issues in additional parallel-safety checks?
- apparent uneven work distribution across the parallel workers, for
large insert data[Antonin]
- Fixed bad Assert in PrepareParallelMode()
- Added missing comment to explain use of GetCurrentCommandId() in
PrepareParallelMode()
- Some variable name shortening in a few places
- Created common function for creation of non-parallel and parallel
ModifyTable paths
- Path count variable changed to bool
- Added FIXME comment to dubious code for creating Gather target-list
from ModifyTable subplan
- Fixed check on returningLists to use NIL instead of NULL[Amit]
- Moved additional parallel-safety checks (for modify case) into
max_parallel_hazard()
- Removed redundant calls to max_parallel_hazard_test()
- Added Asserts to "should never happen" null-attribute cases (and
added WARNING log missing from one case)
- Added comment for use of NoLock in max_parallel_hazard_for_modify()[Vignesh]
- Fixed a couple of typos
- Added a couple of test cases for testing that the same transaction
is used by all parallel workersRegards,
Greg Nancarrow
Fujitsu Australia
On Thu, Jan 21, 2021 at 1:28 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Hi,
For v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT that it currently performs.existing and 'it currently performs' are redundant. You can omit 'that it currently performs'.
OK, but this is very minor.
Minor. For index_expr_max_parallel_hazard_for_modify(),
+ if (keycol == 0) + { + /* Found an index expression */You can check if keycol != 0, continue with the loop. This would save some indent.
Yes I know, but I don't really see any issue with indent (I'm using
4-space tabs).
+ if (index_expr_item == NULL) /* shouldn't happen */ + { + elog(WARNING, "too few entries in indexprs list");I think the warning should be an error since there is assertion ahead of the if statement.
Assertions are normally for DEBUG builds, so the Assert would have no
effect in a production (release) build.
Besides, as I have explained in my reply to previous feedback, the
logging of a WARNING (rather than ERROR) is intentional, because I
want processing to continue (not stop) if ever this very rare
condition was to occur - so that the issue can be dealt with by the
current non-parallel processing (rather than stop dead in the middle
of parallel-safety-checking code). For a DEBUG build, it is handy for
the Assert to immediately alert us to the issue (which could really
only be caused by a database corruption, not bug in the code).
Note that Vignesh originally suggested changing it from
"elog(ERROR,...)" to "elog(WARNING,...)", and I agree with him.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Jan 21, 2021 at 1:50 PM Zhihong Yu <zyu@yugabyte.com> wrote:
For v12-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch:
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
Please wrap long line.
OK.
I thought I ran pg_indent fairly recently, but maybe it chose not to
wrap that line.
+ uint64 *processed_count_space;
If I read the code correctly, it seems it can be dropped (use pei->processed_count directly).
You're right. I'll change it in the next version.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Jan 21, 2021 at 12:47 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
It seems there are some previous comments[1][2] not addressed in current patch.
Just to make sure it's not missed.[1]
/messages/by-id/77e1c06ffb2240838e5fc94ec8dcb7d3@G08CNEXMBPEKD05.g08.fujitsu.local[2]
/messages/by-id/CAA4eK1LMmz58ej5BgVLJ8VsUGd=+KcaA8X=kStORhxpfpODOxg@mail.gmail.com
Thanks for alerting me to those, somehow I completely missed them,
sorry about that.
I'll be sure to investigate and address them in the next version of
the patch I post.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Dec 23, 2020 at 1:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 23, 2020 at 7:52 AM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
I may be wrong, and if I miss sth in previous mails, please give me some
hints.
IMO, serial insertion with underlying parallel SELECT can be
considered for foreign table or temporary table, as the insertions onlyhappened in the leader process.
I don't think we support parallel scan for temporary tables. Can you please
try once both of these operations without Insert being involved? If you
are able to produce a parallel plan without Insert then we can see why it
is not supported with Insert.Sorry, may be I did not express it clearly, I actually means the case when insert's target(not in select part) table is temporary.
And you are right that parallel select is not enabled when temporary table is in select part.I think Select can be parallel for this case and we should support this case.
So I think we're saying that if the target table is a foreign table or
temporary table, it can be regarded as PARALLEL_RESTRICTED, right?
i.e. code-wise:
/*
- * We can't support table modification in parallel-mode if
it's a foreign
- * table/partition (no FDW API for supporting parallel access) or a
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel
access) or a
* temporary table.
*/
if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
RelationUsesLocalBuffers(rel))
{
- table_close(rel, lockmode);
- context->max_hazard = PROPARALLEL_UNSAFE;
- return true;
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
}
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Dec 21, 2020 at 1:50 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
+ + index_oid_list = RelationGetIndexList(rel); ...As memtioned in the comments of RelationGetIndexList:
* we return a copy of the list palloc'd in the caller's context. The caller
* may list_free() the returned list after scanning it.Shall we list_free(index_oid_list) at the end of function ?
Just to avoid potential memory leak.
I think that's a good idea, so I'll make that update in the next
version of the patch.
I do notice, however, that there seems to be quite a few places in the
Postgres code where RelationGetIndexList() is being called without a
corresponding list_free() being called - obviously instead relying on
it being deallocated when the caller's memory-context is destroyed.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Jan 21, 2021 at 12:44 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Dec 23, 2020 at 1:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 23, 2020 at 7:52 AM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
I may be wrong, and if I miss sth in previous mails, please give me some
hints.
IMO, serial insertion with underlying parallel SELECT can be
considered for foreign table or temporary table, as the insertions onlyhappened in the leader process.
I don't think we support parallel scan for temporary tables. Can you please
try once both of these operations without Insert being involved? If you
are able to produce a parallel plan without Insert then we can see why it
is not supported with Insert.Sorry, may be I did not express it clearly, I actually means the case when insert's target(not in select part) table is temporary.
And you are right that parallel select is not enabled when temporary table is in select part.I think Select can be parallel for this case and we should support this case.
So I think we're saying that if the target table is a foreign table or
temporary table, it can be regarded as PARALLEL_RESTRICTED, right?
Yes.
i.e. code-wise:
/* - * We can't support table modification in parallel-mode if it's a foreign - * table/partition (no FDW API for supporting parallel access) or a + * We can't support table modification in a parallel worker if it's a + * foreign table/partition (no FDW API for supporting parallel access) or a * temporary table. */ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE || RelationUsesLocalBuffers(rel)) { - table_close(rel, lockmode); - context->max_hazard = PROPARALLEL_UNSAFE; - return true; + if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)) + { + table_close(rel, lockmode); + return true; + } }
Yeah, these changes look correct to me.
--
With Regards,
Amit Kapila.
Hi
I may be wrong, and if I miss sth in previous mails, please give
me somehints.
IMO, serial insertion with underlying parallel SELECT can be
considered for foreign table or temporary table, as the
insertions onlyhappened in the leader process.
I don't think we support parallel scan for temporary tables. Can
you please try once both of these operations without Insert being
involved? If you are able to produce a parallel plan without
Insert then we can see why it is not supported with Insert.Sorry, may be I did not express it clearly, I actually means the case
when insert's target(not in select part) table is temporary.
And you are right that parallel select is not enabled when temporary
table is in select part.
I think Select can be parallel for this case and we should support this
case.
So I think we're saying that if the target table is a foreign table or
temporary table, it can be regarded as PARALLEL_RESTRICTED, right?
Yes
IMO, PARALLEL_RESTRICTED currently enable parallel select but disable parallel insert.
So, the INSERT only happen in leader worker which seems safe to insert into tempory/foreigh table.
In addition, there are some other restriction about parallel select which seems can be removed:
1.- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
2.- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
If the Insert's target table is the type listed above, Is there some reasons why we can not support parallel select ?
It seems only leader worker will execute the trigger and key-experssion which seems safe.
(If I miss something about it, please let me know)
Best regards,
houzj
So I think we're saying that if the target table is a foreign table or
temporary table, it can be regarded as PARALLEL_RESTRICTED, right?Yes
IMO, PARALLEL_RESTRICTED currently enable parallel select but disable
parallel insert.
So, the INSERT only happen in leader worker which seems safe to insert into
tempory/foreigh table.In addition, there are some other restriction about parallel select which
seems can be removed:1.- Target table has a parallel-unsafe trigger, index expression, column
default
expression or check constraint
2.- Target table is a partitioned table with a parallel-unsafe partition
key
expression or support functionIf the Insert's target table is the type listed above, Is there some reasons
why we can not support parallel select ?
It seems only leader worker will execute the trigger and key-experssion
which seems safe.
(If I miss something about it, please let me know)
So Sorry, please ignore the above, I think of something wrong.
Best regards,
houzj
+ + index_oid_list = RelationGetIndexList(rel); ...As memtioned in the comments of RelationGetIndexList:
* we return a copy of the list palloc'd in the caller's context. The
caller
* may list_free() the returned list after scanning it.Shall we list_free(index_oid_list) at the end of function ?
Just to avoid potential memory leak.I think that's a good idea, so I'll make that update in the next version
of the patch.
I do notice, however, that there seems to be quite a few places in the Postgres
code where RelationGetIndexList() is being called without a corresponding
list_free() being called - obviously instead relying on it being deallocated
when the caller's memory-context is destroyed.
Yes, it will be deallocated when the caller's memory-context is destroyed.
Currently, parallel safety-check check each partition.
I am just a little worried about if there are lots of partition here, it may cause high memory use.
And there is another place like this:
1.
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
It seems we can free the cobin when not used(for the same reason above).
What do you think ?
Best regards,
houzj
On Fri, Jan 22, 2021 at 12:08 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
I think that's a good idea, so I'll make that update in the next version
of the patch.
I do notice, however, that there seems to be quite a few places in the Postgres
code where RelationGetIndexList() is being called without a corresponding
list_free() being called - obviously instead relying on it being deallocated
when the caller's memory-context is destroyed.Yes, it will be deallocated when the caller's memory-context is destroyed.
Currently, parallel safety-check check each partition.
I am just a little worried about if there are lots of partition here, it may cause high memory use.And there is another place like this:
1. + conbin = TextDatumGetCString(val); + check_expr = stringToNode(conbin);It seems we can free the cobin when not used(for the same reason above).
What do you think ?
Yes, I think you're right, we should pfree conbin after converting to
Node, to minimize memory usage.
Again, it's interesting that existing Postgres code, when looping
through all of the constraints, doesn't do this.
Hmmm. I'm wondering if there is a performance reason behind this -
avoiding multiple calls to pfree() and just relying on it to be
deallocated in one hit, when the memory context is destroyed.
Anyway, perhaps the concerns of many partitions and the recursive
nature of these checks overrides that, because, as you say, possible
high memory usage.
Regards,
Greg Nancarrow
Fujitsu Australia
Hi
I took a look at v12-0001 patch, here are some comments:
1.
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == 0 ||
+ initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == 0 ?
+ PROPARALLEL_SAFE : initial_max_parallel_hazard;
I am not quiet sure when will " max_parallel_hazard == 0"
Does it means the case max_parallel_hazard_context not initialized ?
2.
Some tiny code style suggestions
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
How about :
if (con->contype != CONSTRAINT_CHECK)
continue;
3.
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
Like 2, how about:
If (keycol != 0)
Continue;
4.
+ ListCell *index_expr_item = list_head(index_info->ii_Expressions);
...
+ index_expr = (Node *) lfirst(index_expr_item);
+ index_expr = (Node *) expression_planner((Expr *) index_expr);
It seems BuildIndexInfo has already called eval_const_expressions for ii_Expressions,
Like the flow: BuildIndexInfo--> RelationGetIndexExpressions--> eval_const_expressions
So, IMO, we do not need to call expression_planner for the expr again.
And there seems another solution for this:
In the patch, We only use the { ii_Expressions , ii_NumIndexAttrs , ii_IndexAttrNumbers } from the IndexInfo,
which seems can get from "Relation-> rd_index".
Based on above, May be we do not need to call BuildIndexInfo to build the IndexInfo.
It can avoid some unnecessary cost.
And in this solution we do not need to remove expression_planner.
What do you think ?
Best regards,
houzj
On Thu, Jan 21, 2021 at 7:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
i.e. code-wise:
/* - * We can't support table modification in parallel-mode if it's a foreign - * table/partition (no FDW API for supporting parallel access) or a + * We can't support table modification in a parallel worker if it's a + * foreign table/partition (no FDW API for supporting parallel access) or a * temporary table. */ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE || RelationUsesLocalBuffers(rel)) { - table_close(rel, lockmode); - context->max_hazard = PROPARALLEL_UNSAFE; - return true; + if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)) + { + table_close(rel, lockmode); + return true; + } }Yeah, these changes look correct to me.
Unfortunately, this change results in a single test failure in the
"with" tests when "force_parallel_mode=regress" is in effect.
I have reproduced the problem, by extracting relevant SQL from those
tests, as follows:
CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;
SELECT * FROM bug6051;
CREATE TEMP TABLE bug6051_2 (i int);
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD
INSERT INTO bug6051_2
SELECT NEW.i;
WITH t1 AS ( DELETE FROM bug6051 RETURNING * )
INSERT INTO bug6051 SELECT * FROM t1;
ERROR: cannot delete tuples during a parallel operation
Note that prior to the patch, all INSERTs were regarded as
PARALLEL_UNSAFE, so this problem obviously didn't occur.
I believe this INSERT should be regarded as PARALLEL_UNSAFE, because
it contains a modifying CTE.
However, for some reason, the INSERT is not regarded as having a
modifying CTE, so instead of finding it PARALLEL_UNSAFE, it falls into
the parallel-safety-checks and is found to be PARALLEL_RESTRICTED:
The relevant code in standard_planner() is:
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}
When I debugged this (transformWithClause()), the WITH clause was
found to contain a modifying CTE and for the INSERT
query->hasModifyingCTE was set true.
But somehow in the re-writer code, this got lost.
Bug?
Ideas?
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Jan 22, 2021 at 1:16 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi
I took a look at v12-0001 patch, here are some comments:
1. + /* + * Setup the context used in finding the max parallel-mode hazard. + */ + Assert(initial_max_parallel_hazard == 0 || + initial_max_parallel_hazard == PROPARALLEL_SAFE || + initial_max_parallel_hazard == PROPARALLEL_RESTRICTED); + context.max_hazard = initial_max_parallel_hazard == 0 ? + PROPARALLEL_SAFE : initial_max_parallel_hazard;I am not quiet sure when will " max_parallel_hazard == 0"
Does it means the case max_parallel_hazard_context not initialized ?
That function doesn't accept a "max_parallel_hazard_context". It
accepts an initial "max_parallel_hazard" value (char).
The "0" value is just a way of specifying "use the default"
(PROPARALLEL_SAFE). It is not currently used, since currently we just
always pass the "context.max_parallel_hazard" value resulting from the
previous parallel-safety checks for SELECT (and otherwise don't call
that function anywhere else).
2.
Some tiny code style suggestions+ if (con->contype == CONSTRAINT_CHECK) + { + char *conbin; + Datum val; + bool isnull; + Expr *check_expr;How about :
if (con->contype != CONSTRAINT_CHECK)
continue;3. + if (keycol == 0) + { + /* Found an index expression */ + + Node *index_expr;Like 2, how about:
If (keycol != 0)
Continue;
This is really a programmer style preference (plenty of discussions on
the internet about it), but it can be argued that use of "continue"
here is not quite as clear as the explicit "if" condition, especially
in this very simple one-condition case.
I'm inclined to leave it as is.
4. + ListCell *index_expr_item = list_head(index_info->ii_Expressions); ... + index_expr = (Node *) lfirst(index_expr_item); + index_expr = (Node *) expression_planner((Expr *) index_expr);It seems BuildIndexInfo has already called eval_const_expressions for ii_Expressions,
Like the flow: BuildIndexInfo--> RelationGetIndexExpressions--> eval_const_expressionsSo, IMO, we do not need to call expression_planner for the expr again.
And there seems another solution for this:
In the patch, We only use the { ii_Expressions , ii_NumIndexAttrs , ii_IndexAttrNumbers } from the IndexInfo,
which seems can get from "Relation-> rd_index".Based on above, May be we do not need to call BuildIndexInfo to build the IndexInfo.
It can avoid some unnecessary cost.
And in this solution we do not need to remove expression_planner.
OK, maybe this is a good idea, but I do recall trying to minimize this
kind of processing before, but there were cases which broke it.
Have you actually tried your idea and run all regression tests and
verified that they passed?
In any case, I'll look into it.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Jan 22, 2021 at 8:29 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Jan 21, 2021 at 7:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
i.e. code-wise:
/* - * We can't support table modification in parallel-mode if it's a foreign - * table/partition (no FDW API for supporting parallel access) or a + * We can't support table modification in a parallel worker if it's a + * foreign table/partition (no FDW API for supporting parallel access) or a * temporary table. */ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE || RelationUsesLocalBuffers(rel)) { - table_close(rel, lockmode); - context->max_hazard = PROPARALLEL_UNSAFE; - return true; + if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)) + { + table_close(rel, lockmode); + return true; + } }Yeah, these changes look correct to me.
Unfortunately, this change results in a single test failure in the
"with" tests when "force_parallel_mode=regress" is in effect.I have reproduced the problem, by extracting relevant SQL from those
tests, as follows:CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;
SELECT * FROM bug6051;
CREATE TEMP TABLE bug6051_2 (i int);
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD
INSERT INTO bug6051_2
SELECT NEW.i;
WITH t1 AS ( DELETE FROM bug6051 RETURNING * )
INSERT INTO bug6051 SELECT * FROM t1;
ERROR: cannot delete tuples during a parallel operationNote that prior to the patch, all INSERTs were regarded as
PARALLEL_UNSAFE, so this problem obviously didn't occur.
I believe this INSERT should be regarded as PARALLEL_UNSAFE, because
it contains a modifying CTE.
However, for some reason, the INSERT is not regarded as having a
modifying CTE, so instead of finding it PARALLEL_UNSAFE, it falls into
the parallel-safety-checks and is found to be PARALLEL_RESTRICTED:The relevant code in standard_planner() is:
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}When I debugged this (transformWithClause()), the WITH clause was
found to contain a modifying CTE and for the INSERT
query->hasModifyingCTE was set true.
But somehow in the re-writer code, this got lost.
Bug?
Ideas?
How it behaves when the table in the above test is a non-temp table
with your patch? If it leads to the same error then we can at least
conclude that this is a generic problem and nothing specific to temp
tables.
--
With Regards,
Amit Kapila.
On Fri, Jan 22, 2021 at 1:16 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
4. + ListCell *index_expr_item = list_head(index_info->ii_Expressions); ... + index_expr = (Node *) lfirst(index_expr_item); + index_expr = (Node *) expression_planner((Expr *) index_expr);It seems BuildIndexInfo has already called eval_const_expressions for ii_Expressions,
Like the flow: BuildIndexInfo--> RelationGetIndexExpressions--> eval_const_expressionsSo, IMO, we do not need to call expression_planner for the expr again.
Thanks. You are right. I debugged it, and found that BuildIndexInfo-->
RelationGetIndexExpressions executes the same expression evaluation
code as expression_planner().
So I'll remove the redundant call to expression_planner() here.
And there seems another solution for this:
In the patch, We only use the { ii_Expressions , ii_NumIndexAttrs , ii_IndexAttrNumbers } from the IndexInfo,
which seems can get from "Relation-> rd_index".Based on above, May be we do not need to call BuildIndexInfo to build the IndexInfo.
It can avoid some unnecessary cost.
And in this solution we do not need to remove expression_planner.
Hmmm, when I debugged my simple test case, I found rel->rd_index was
NULL, so it seems that the call to BuildIndexInfo is needed.
(have I understood your suggestion correctly?)
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Jan 22, 2021 at 4:49 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Unfortunately, this change results in a single test failure in the
"with" tests when "force_parallel_mode=regress" is in effect.I have reproduced the problem, by extracting relevant SQL from those
tests, as follows:CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;
SELECT * FROM bug6051;
CREATE TEMP TABLE bug6051_2 (i int);
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD
INSERT INTO bug6051_2
SELECT NEW.i;
WITH t1 AS ( DELETE FROM bug6051 RETURNING * )
INSERT INTO bug6051 SELECT * FROM t1;
ERROR: cannot delete tuples during a parallel operationNote that prior to the patch, all INSERTs were regarded as
PARALLEL_UNSAFE, so this problem obviously didn't occur.
I believe this INSERT should be regarded as PARALLEL_UNSAFE, because
it contains a modifying CTE.
However, for some reason, the INSERT is not regarded as having a
modifying CTE, so instead of finding it PARALLEL_UNSAFE, it falls into
the parallel-safety-checks and is found to be PARALLEL_RESTRICTED:The relevant code in standard_planner() is:
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}When I debugged this (transformWithClause()), the WITH clause was
found to contain a modifying CTE and for the INSERT
query->hasModifyingCTE was set true.
But somehow in the re-writer code, this got lost.
Bug?
Ideas?How it behaves when the table in the above test is a non-temp table
with your patch? If it leads to the same error then we can at least
conclude that this is a generic problem and nothing specific to temp
tables.
Oh, I don't believe that this has anything to do with TEMP tables -
it's just that when I relaxed the parallel-safety level on TEMP
tables, it exposed the CTE issue in this test case because it just
happens to use a TEMP table.
Having said that, when I changed that test code to not use a TEMP
table, an Assert fired in the planner code and caused the backend to
abort.
It looks like I need to update the following Assert in the planner
code (unchanged by the current patch) in order to test further - but
this Assert only fired because the commandType was CMD_DELETE, which
SHOULD have been excluded by the "hasModifyingCTE" test on the parent
INSERT, which is what I'm saying is strangely NOT getting set.
/*
* Generate partial paths for final_rel, too, if outer query levels might
* be able to make use of them.
*/
if (final_rel->consider_parallel && root->query_level > 1 &&
!limit_needed(parse))
{
Assert(!parse->rowMarks && parse->commandType == CMD_SELECT);
foreach(lc, current_rel->partial_pathlist)
{
Path *partial_path = (Path *) lfirst(lc);
add_partial_path(final_rel, partial_path);
}
}
Once the Assert above is changed to not test the commandType, the same
error occurs as before.
This appears to possibly be some kind of bug in which hasModifyingCTE
is not getting set, at least in this particular INSERT case, but in
the current Postgres code it doesn't matter because all INSERTs are
considered parallel-unsafe, so won't be executed in parallel-mode
anyway.
I notice that if I execute "SELECT * from t1" instead of "INSERT INTO
bug6051 SELECT * from t1", then "hasModifyingCTE" is getting set to
true for the query, and thus is always considered parallel-unsafe.
Regards,
Greg Nancarrow
Fujitsu Australia
And there seems another solution for this:
In the patch, We only use the { ii_Expressions , ii_NumIndexAttrs ,
ii_IndexAttrNumbers } from the IndexInfo, which seems can get from"Relation-> rd_index".
Based on above, May be we do not need to call BuildIndexInfo to build
the IndexInfo.
It can avoid some unnecessary cost.
And in this solution we do not need to remove expression_planner.Hmmm, when I debugged my simple test case, I found rel->rd_index was NULL,
so it seems that the call to BuildIndexInfo is needed.
(have I understood your suggestion correctly?)
Hi greg,
Thanks for debugging this.
May be I missed something. I am not sure about the case when rel->rd_index was NULL.
Because, In function BuildIndexInfo, it seems does not have NULL-check for index->rd_index.
Like the following:
----
BuildIndexInfo(Relation index)
{
IndexInfo *ii;
Form_pg_index indexStruct = index->rd_index;
int i;
int numAtts;
/* check the number of keys, and copy attr numbers into the IndexInfo */
numAtts = indexStruct->indnatts;
----
And the patch do not have NULL-check for index->rd_index too.
So I thought we can assume index->rd_index is not null, but it seems I may missed something ?
Can you please share the test case with me ?
I use the following code to replace the call of BuildIndexInfo.
And the installcheck passed.
Example:
+ Form_pg_index indexStruct = index_rel->rd_index;
+ List *ii_Expressions = RelationGetIndexExpressions(index_rel);
+ int ii_NumIndexAttrs = indexStruct->indnatts;
+ AttrNumber ii_IndexAttrNumbers[INDEX_MAX_KEYS];
+ for (int i = 0; i < ii_NumIndexAttrs; i++)
+ ii_IndexAttrNumbers[i] = indexStruct->indkey.values[i];
Best regards,
houzj
On Fri, Jan 22, 2021 at 6:21 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi greg,
Thanks for debugging this.
May be I missed something. I am not sure about the case when rel->rd_index was NULL.
Because, In function BuildIndexInfo, it seems does not have NULL-check for index->rd_index.
Like the following:
----
BuildIndexInfo(Relation index)
{
IndexInfo *ii;
Form_pg_index indexStruct = index->rd_index;
int i;
int numAtts;/* check the number of keys, and copy attr numbers into the IndexInfo */
numAtts = indexStruct->indnatts;
----And the patch do not have NULL-check for index->rd_index too.
So I thought we can assume index->rd_index is not null, but it seems I may missed something ?Can you please share the test case with me ?
I use the following code to replace the call of BuildIndexInfo.
And the installcheck passed.Example: + Form_pg_index indexStruct = index_rel->rd_index; + List *ii_Expressions = RelationGetIndexExpressions(index_rel); + int ii_NumIndexAttrs = indexStruct->indnatts; + AttrNumber ii_IndexAttrNumbers[INDEX_MAX_KEYS];+ for (int i = 0; i < ii_NumIndexAttrs; i++) + ii_IndexAttrNumbers[i] = indexStruct->indkey.values[i];
Sorry, I was looking at rel->rd_index, not index_rel->rd_index, my fault.
Your code looks OK. I've taken it and reduced some of the lines and
got rid of the C99-only intermingled variable declarations (see
https://www.postgresql.org/docs/13/source-conventions.html).
The changes are below.
The regression tests all pass, so should be OK (my test case was taken
from insert_parallel regression tests).
Thanks for your help.
- Oid index_oid = lfirst_oid(lc);
- Relation index_rel;
- IndexInfo *index_info;
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
index_rel = index_open(index_oid, lockmode);
- index_info = BuildIndexInfo(index_rel);
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
- if (index_info->ii_Expressions != NIL)
+ if (ii_Expressions != NIL)
{
int i;
- ListCell *index_expr_item =
list_head(index_info->ii_Expressions);
+ ListCell *index_expr_item = list_head(ii_Expressions);
- for (i = 0; i < index_info->ii_NumIndexAttrs; i++)
+ for (i = 0; i < indexStruct->indnatts; i++)
{
- int keycol = index_info->ii_IndexAttrNumbers[i];
+ int keycol = indexStruct->indkey.values[i];
if (keycol == 0)
{
@@ -912,7 +914,7 @@ index_expr_max_parallel_hazard_for_modify(Relation rel,
return true;
}
- index_expr_item =
lnext(index_info->ii_Expressions, index_expr_item);
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
}
}
Regards,
Greg Nancarrow
Fujitsu Australia
Hello Greg-san,
Initially, some miner comments:
(1)
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
This can read "In INSERT INTO...SELECT case, like other existing cases, only the leader backend writes into a completely new table." The reality is that workers as well as the leader can write into an empty or non-empty table in parallel, isn't it?
(2)
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
- ((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ ((relation)->rd_islocaltemp)
How is this correct? At least, this change would cause a transaction that creates a new relation acquire an unnecessary lock. I'm not sure if that overhead is worth worrying about (perhaps not, I guess). But can we still check >rd_createSubid in non-parallel mode? If we adopt the above change, the comments at call sites need modification - "new or temp relation" becomes "temp relations".
(3)
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
...
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
The last line can just be as follows, according to the Assert():
+ pstmt->commandType = estate->es_plannedstmt->commandType);
(4)
@@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelMode(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode, based on command-type.
*/
void
-PrepareParallelMode(CmdType commandType)
+PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
I think we can eliminate the second argument of PrepareParallelMode() and the new code in ExecutePlan(). PrepareParallelMode() can use !IsParallelWorker() in the if condition, because the caller is either a would-be parallel leader or a parallel worker.
BTW, why do we want to add PrepareParallelMode() separately from EnterParallelMode()? Someone who will read other call sites of EnterParallelMode() (index build, VACUUM) may be worried that PrepareParallelMode() call is missing there. Can we just add an argument to EnterParallelMode()? Other call sites can use CMD_UNKNOWN or CMD_UTILITY, if we want to use CMD_XX.
Regards
Takayuki Tsunakawa
On Fri, Jan 22, 2021 at 7:52 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
(1) - * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE - * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader - * backend writes into a completely new table. In the future, we can - * extend it to allow workers to write into the table. However, to allow - * parallel updates and deletes, we have to solve other problems, - * especially around combo CIDs.) + * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT + * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as + * of now, only the leader backend writes into a completely new table. InThis can read "In INSERT INTO...SELECT case, like other existing cases, only the leader backend writes into a completely new table." The reality is that workers as well as the leader can write into an empty or non-empty table in parallel, isn't it?
Yes, you're right the wording is not right (and I don't really like
the wording used before the patch).
Perhaps it could say:
(Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
of now, other than in the case of INSERT INTO...SELECT, only the leader backend
writes into a completely new table. In the future, we can extend it to
allow workers for the
other commands to write into the table. However, to allow parallel
updates and deletes, we
have to solve other problems, especially around combo CIDs.)
Of course, this will need further updating when parallel CREATE TABLE
AS etc. is implemented ...
(2) /* * RELATION_IS_LOCAL - * If a rel is either temp or newly created in the current transaction, - * it can be assumed to be accessible only to the current backend. - * This is typically used to decide that we can skip acquiring locks. + * If a rel is temp, it can be assumed to be accessible only to the + * current backend. This is typically used to decide that we can + * skip acquiring locks. * * Beware of multiple eval of argument */ #define RELATION_IS_LOCAL(relation) \ - ((relation)->rd_islocaltemp || \ - (relation)->rd_createSubid != InvalidSubTransactionId) + ((relation)->rd_islocaltemp)How is this correct? At least, this change would cause a transaction that creates a new relation acquire an unnecessary lock. I'm not sure if that overhead is worth worrying about (perhaps not, I guess). But can we still check >rd_createSubid in non-parallel mode? If we adopt the above change, the comments at call sites need modification - "new or temp relation" becomes "temp relations".
The problem is, with the introduction of parallel INSERT, it's no
longer the case that newly-created tables can't be accessed by anyone
else in the same transaction - now, a transaction can include parallel
workers, inserting into the table concurrently. Without changing that
macro, things fail with a very obscure message (e.g. ERROR:
unexpected data beyond EOF in block 5 of relation base/16384/16388)
and it takes days to debug what the cause of it is.
Maybe updating the macro to still check rd_createSubid in non-parallel
mode is a good idea - I'll need to try it.
Other than that, each and every usage of RELATION_IS_LOCAL would need
to be closely examined, to see if it could be within a parallel
INSERT.
(3) @@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate) ... - pstmt->commandType = CMD_SELECT; + Assert(estate->es_plannedstmt->commandType == CMD_SELECT || + IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType)); + pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;The last line can just be as follows, according to the Assert():
+ pstmt->commandType = estate->es_plannedstmt->commandType);
No, that's not right. I did that originally and it failed in some
cases (try changing it and then run the regression tests and you'll
see).
The commandType of the es_plannedstmt might be CMD_INSERT but the one
in the plan might be CMD_SELECT (for the underlying SELECT).
(4) @@ -1527,7 +1528,9 @@ ExecutePlan(EState *estate, estate->es_use_parallel_mode = use_parallel_mode; if (use_parallel_mode) { - PrepareParallelMode(estate->es_plannedstmt->commandType); + bool isParallelModifyLeader = IsA(planstate, GatherState) && IsA(outerPlanState(planstate), ModifyTableState); + + PrepareParallelMode(estate->es_plannedstmt->commandType, isParallelModifyLeader); EnterParallelMode(); }@@ -1021,12 +1039,25 @@ IsInParallelMode(void) * Prepare for entering parallel mode, based on command-type. */ void -PrepareParallelMode(CmdType commandType) +PrepareParallelMode(CmdType commandType, bool isParallelModifyLeader) { if (IsModifySupportedInParallelMode(commandType)) { Assert(!IsInParallelMode());+ if (isParallelModifyLeader) + { + /* + * Set currentCommandIdUsed to true, to ensure that the current + * CommandId (which will be used by the parallel workers) won't + * change during this parallel operation, as starting new + * commands in parallel-mode is not currently supported. + * See related comments in GetCurrentCommandId and + * CommandCounterIncrement. + */ + (void) GetCurrentCommandId(true); + }I think we can eliminate the second argument of PrepareParallelMode() and the new code in ExecutePlan(). PrepareParallelMode() can use !IsParallelWorker() in the if condition, because the caller is either a would-be parallel leader or a parallel worker.
You could, but I'm not sure it would make the code easier to read,
especially for those who don't know !isParallelWorker() means it's a
parallel leader.
BTW, why do we want to add PrepareParallelMode() separately from EnterParallelMode()? Someone who will read other call sites of EnterParallelMode() (index build, VACUUM) may be worried that PrepareParallelMode() call is missing there. Can we just add an argument to EnterParallelMode()? Other call sites can use CMD_UNKNOWN or CMD_UTILITY, if we want to use CMD_XX.
I really can't see a problem. PrepareParallelMode() is only needed
prior to execution of a parallel plan, so it's not needed for "other
call sites" using EnterParallelMode().
Perhaps the name can be changed to disassociate it from generic
EnterParallelMode() usage. So far, I've only thought of long names
like: PrepareParallelModePlanExec().
Ideas?
Regards,
Greg Nancarrow
Fujitsu Australia
.
Hello Greg-san,
Second group of comments (I'll reply to (1) - (4) later):
(5)
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
...
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
Now that we're trying to allow parallel writes (INSERT), we should:
* use ExecCheckXactReadOnly() solely for checking read-only transactions, as the function name represents. That is, move the call to PreventCommandIfParallelMode() up to standard_ExecutorStart().
* Update the comment above the call to ExecCheckXactReadOnly().
(6)
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
...
+ else
+ {
+ pei->processed_count = NULL;
+ }
The braces can be deleted.
(7)
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsed()), otherwise forbid
+ * setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
What happens without these changes? If this kind of change is really necessary, it seems more natural to pass currentCommandIdUsed together with currentCommandId through SerializeTransactionState() and StartParallelWorkerTransaction(), instead of the above changes.
As an aside, SetCurrentCommandIdUsed() in the comment should be SetCurrentCommandIdUsedForWorker().
(8)
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
Here, RI_TRIGGER_FK should instead be RI_TRIGGER_PK, because RI_TRIGGER_FK triggers do not generate command IDs. See RI_FKey_check() which is called in RI_TRIGGER_FK case. In there, ri_PerformCheck() is called with the detectNewRows argument set to false, which causes CommandCounterIncrement() to not be called.
Plus, tables that have RI_TRIGGER_PK should allow parallel INSERT in a parallel-safe manner, because those triggers only fire for UPDATE and DELETE. So, for the future parallel UPDATE/DELETE support, the above check should be performed in UPDATE and DELETE cases.
(In a data warehouse, fact tables, which store large amounts of historical data, typically have foreign keys to smaller dimension tables. Thus, it's important to allow parallel INSERTs on tables with foreign keys.)
Regards
Takayuki Tsunakawa
Hi,
After doing some test to cover the code path in the PATCH 0001.
I have some suggestions for the 0002 testcase.
(1)
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
The testcase seems does not cover the above code(test when the table have parallel unsafe expression in the partition key).
Personally, I use the following sql to cover this:
-----
create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
-----
(2)
I noticed that most of testcase test both (parallel safe/unsafe/restricted).
But the index expression seems does not test the parallel restricted.
How about add a testcase like:
-----
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
--
-- Test INSERT with parallel-restricted index expression
-- (should create a parallel plan)
--
explain (costs off) insert into names4 select * from names;
-----
(3)
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ if (rel_max_parallel_hazard_for_modify(pdesc->oids[i],
+ command_type,
+ context,
+ AccessShareLock))
+ {
+ table_close(rel, lockmode);
+ return true;
+ }
+ }
It seems we do not have a testcase to test (some parallel unsafe expression or.. in partition)
Hoe about add one testcase to test parallel unsafe partition ?
Best regards,
houzj
On Mon, Jan 25, 2021 at 10:23 AM tsunakawa.takay@fujitsu.com <
tsunakawa.takay@fujitsu.com> wrote:
Hello Greg-san,
Second group of comments (I'll reply to (1) - (4) later):
(5) @@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt) ... - if (plannedstmt->commandType != CMD_SELECT ||
plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT && +
!IsModifySupportedInParallelMode(plannedstmt->commandType)) ||
plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *)
plannedstmt));
}
Now that we're trying to allow parallel writes (INSERT), we should:
* use ExecCheckXactReadOnly() solely for checking read-only transactions,
as the function name represents. That is, move the call to
PreventCommandIfParallelMode() up to standard_ExecutorStart().
* Update the comment above the call to ExecCheckXactReadOnly().
Hmmm, I not so sure. The patch changes just make the existing test for
calling PreventCommandIfParallelMode() a bit more restrictive, to exclude
the Parallel INSERT case. So the code previously wasn't just checking
read-only transactions anyway, so it's not as if the patch has changed
something fundamental in this function. And by moving the
PreventCommandIfParallelMode() call to a higher level, then you're making a
change to the existing order of error-handling (as ExecCheckXactReadOnly()
is calling PreventCommandIfReadOnly() based on a few other range-table
conditions, prior to testing whether to call
PreventCommandIfParallelMode()). I don't want to introduce a bug by making
the change that you're suggesting.
(6)
@@ -764,6 +777,22 @@ ExecInitParallelPlan(PlanState *planstate, EState
*estate,
... + else + { + pei->processed_count = NULL; + }The braces can be deleted.
Yes they can be deleted, and I guess I will, but for the record, I
personally prefer the explicit brackets, even if just one line, because:
- if more code ever needs to be added to the else, you'll need to add
brackets anyway (and newbies might add extra lines tabbed in, thinking it's
part of the else block ...).
- I think it looks better and slightly easier to read, especially when
there's a mix of cases with multiple code lines and single code lines
Of course, these kind of things could be debated forever, but I don't think
it's such a big deal.
(7)
@@ -1400,6 +1439,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver,
instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT ||
IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation)) + { + /* + * Record that the CurrentCommandId is used, at the start
of the
+ * parallel operation. + */ + SetCurrentCommandIdUsedForWorker(); + } + /* Setting debug_query_string for individual workers */ debug_query_string = queryDesc->sourceText;@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used) if (used) { /* - * Forbid setting currentCommandIdUsed in a parallel
worker, because
- * we have no provision for communicating this back to
the leader. We
- * could relax this restriction when currentCommandIdUsed
was already
- * true at the start of the parallel operation. + * If in a parallel worker, only allow setting
currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of
the parallel
+ * operation (by way of SetCurrentCommandIdUsed()),
otherwise forbid
+ * setting currentCommandIdUsed because we have no
provision for
+ * communicating this back to the leader. Once
currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be
changed,
+ * because CommandCounterIncrement() then prevents any
attempted
+ * increment of the current commandId. */ - Assert(!IsParallelWorker()); + Assert(!(IsParallelWorker() && !currentCommandIdUsed)); currentCommandIdUsed = true; } return currentCommandId;What happens without these changes?
The change to the above comment explains why the change is needed.
Without these changes, a call in a parallel worker to GetCurrentCommandId()
will result in an Assert being fired because (prior to the patch)
currentCommandIdUsed is forbidden to be set in a parallel worker, and
calling GetCurrentCommandId(true) (to signify the intent to use the
returned CommandId to mark inserted/updated/deleted tuples) will result in
currentCommandIdUsed being set to true.
So it is clear that this cannot remain the same, in order to support
Parallel INSERT by workers.
So for each worker, the patch sets "currentCommandIdUsed" to true at the
start of the parallel operation (using SetCurrentCommandIdUsedForWorker())
and the Assert condition in GetCurrentCommandId() is tweaked to fire the
Assert if GetCurrentCommandId(true) is called in a parallel worker when
currentCommandIdUsed is false;
To me, this makes perfect sense.
If this kind of change is really necessary, it seems more natural to pass
currentCommandIdUsed together with currentCommandId through
SerializeTransactionState() and StartParallelWorkerTransaction(), instead
of the above changes.
No, I don't agree with that. That approach doesn't sound right to me at all.
All the patch really changes is WHERE "currentCurrentIdUsed" can be set for
a parallel worker - now it is only allowed to be set to true at the start
of the parallel operation for each worker, and the Assert (which is just a
sanity check) is updated to ensure that for workers, it can only be set
true at that time. That's all it does. It's completely consistent with the
old comment that said "We could relax this restriction when
currentCommandIdUsed was already true at the start of the parallel
operation" - that's what we are now doing with the patch.
As an aside, SetCurrentCommandIdUsed() in the comment should be
SetCurrentCommandIdUsedForWorker().
Thanks, I'll fix that in the comments.
(8) + /* + * If the trigger type is RI_TRIGGER_FK, this indicates a
FK exists in
+ * the relation, and this would result in creation of new
CommandIds
+ * on insert/update/delete and this isn't supported in a
parallel
+ * worker (but is safe in the parallel leader). + */ + trigtype = RI_FKey_trigger_type(trigger->tgfoid); + if (trigtype == RI_TRIGGER_FK) + { + if
(max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true; + }Here, RI_TRIGGER_FK should instead be RI_TRIGGER_PK, because
RI_TRIGGER_FK triggers do not generate command IDs. See RI_FKey_check()
which is called in RI_TRIGGER_FK case. In there, ri_PerformCheck() is
called with the detectNewRows argument set to false, which causes
CommandCounterIncrement() to not be called.
Hmmm, I'm not sure that you have read and interpreted the patch code
correctly.
The existence of a RI_TRIGGER_FK trigger indicates the table has a foreign
key, and an insert into such a table will generate a new commandId (so we
must avoid that, as we don't currently have the technology to support
sharing of new command IDs across the participants in the parallel
operation). This is what the code comment says, It does not say that such a
trigger generates a new command ID.
See Amit's updated comment here:
https://github.com/postgres/postgres/commit/0d32511eca5aec205cb6b609638ea67129ef6665
In addition, the 2nd patch has an explicit test case for this (testing
insert into a table that has a FK).
If you have a test case that breaks the existing patch, please let me know.
Regards,
Greg Nancarrow
Fujitsu Australia
From: Greg Nancarrow <gregn4422@gmail.com>
(1)
Yes, you're right the wording is not right (and I don't really like
the wording used before the patch).Perhaps it could say:
(Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
of now, other than in the case of INSERT INTO...SELECT, only the leader
backend
writes into a completely new table. In the future, we can extend it to
allow workers for the
other commands to write into the table. However, to allow parallel
updates and deletes, we
have to solve other problems, especially around combo CIDs.)
That looks good to me, thanks.
(4)
You could, but I'm not sure it would make the code easier to read,
especially for those who don't know !isParallelWorker() means it's a
parallel leader.
...
I really can't see a problem. PrepareParallelMode() is only needed
prior to execution of a parallel plan, so it's not needed for "other
call sites" using EnterParallelMode().
My frank first impressions were (and are):
* Why do we have to call a separate function for preparation despite the actual entering follows immediately? We can do necessary preparation in the entering function.
* Those who read the parallel index build and parallel VACUUM code for the first time might be startled at the missing PrepareParallelMode() call: "Oh, EnterParallelMode() is called without preparation unlike the other site I saw the other day. Isn't this a but?"
Perhaps the name can be changed to disassociate it from generic
EnterParallelMode() usage. So far, I've only thought of long names
like: PrepareParallelModePlanExec().
Ideas?
What PrepareParallelMode() handles is the XID and command ID, which are managed by access/transam/ module and are not executor-specific. It's natural (or at least not unnatural) that EnterParallelMode() prepares them, because EnterParallelMode() is part of access/transam/.
Regards
Takayuki Tsunakawa
On Mon, Jan 25, 2021 at 2:22 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com>
wrote:
Hi,
After doing some test to cover the code path in the PATCH 0001.
I have some suggestions for the 0002 testcase.(1) + /* Check parallel-safety of any expressions in the partition key */ + if (get_partition_col_attnum(pkey, i) == 0) + { + Node *check_expr = (Node *) lfirst(partexprs_item); + + if (max_parallel_hazard_walker(check_expr, context)) + { + table_close(rel, lockmode); + return true; + }The testcase seems does not cover the above code(test when the table have
parallel unsafe expression in the partition key).Personally, I use the following sql to cover this:
-----
create table partkey_unsafe_key_expr_t (a int4, b name) partition by range
((fullname_parallel_unsafe('',a::varchar)));
explain (costs off) insert into partkey_unsafe_key_expr_t select unique1,
stringu1 from tenk1;
-----
Thanks. It looks like that test case was accidently missed (since the
comment said to test the index expressions, but it actually tested the
support functions).
I'll update the test code (and comments) accordingly, using your
suggestion.
(2)
I noticed that most of testcase test both (parallel
safe/unsafe/restricted).
But the index expression seems does not test the parallel restricted.
How about add a testcase like:
-----
create or replace function fullname_parallel_restricted(f text, l text)
returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4
(fullname_parallel_restricted(first_name, last_name));--
-- Test INSERT with parallel-restricted index expression
-- (should create a parallel plan)
--
explain (costs off) insert into names4 select * from names;
-----
Thanks, looks like that test case is missing, I'll add it as you suggest.
(3) + /* Recursively check each partition ... */ + pdesc = RelationGetPartitionDesc(rel); + for (i = 0; i < pdesc->nparts; i++) + { + if (rel_max_parallel_hazard_for_modify(pdesc->oids[i], + command_type, + context, + AccessShareLock)) + { + table_close(rel, lockmode); + return true; + } + }It seems we do not have a testcase to test (some parallel unsafe
expression or.. in partition)
Hoe about add one testcase to test parallel unsafe partition ?
OK, I have to create a more complex table to test those other potential
parallel-safety issues of partitions (other than what was tested before the
recursive call, or support functions and expression in index key), but
since it's a recursive call, invoking code that's already been tested, I
would not anticipate any problems.
Thanks,
Greg Nancarrow
Fujitsu Australia
On Mon, Jan 25, 2021 at 4:37 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
(4)
You could, but I'm not sure it would make the code easier to read,
especially for those who don't know !isParallelWorker() means it's a
parallel leader....
I really can't see a problem. PrepareParallelMode() is only needed
prior to execution of a parallel plan, so it's not needed for "other
call sites" using EnterParallelMode().My frank first impressions were (and are):
* Why do we have to call a separate function for preparation despite the actual entering follows immediately? We can do necessary preparation in the entering function.
* Those who read the parallel index build and parallel VACUUM code for the first time might be startled at the missing PrepareParallelMode() call: "Oh, EnterParallelMode() is called without preparation unlike the other site I saw the other day. Isn't this a but?"
Perhaps the name can be changed to disassociate it from generic
EnterParallelMode() usage. So far, I've only thought of long names
like: PrepareParallelModePlanExec().
Ideas?What PrepareParallelMode() handles is the XID and command ID, which are managed by access/transam/ module and are not executor-specific. It's natural (or at least not unnatural) that EnterParallelMode() prepares them, because EnterParallelMode() is part of access/transam/.
EnterParallelMode() is part of a generic interface for execution of a
parallel operation, and EnterParallelMode() is called in several
different places to enter parallel mode prior to execution of
different parallel operations. At the moment it is assumed that
EnterParallelMode() just essentially sets a flag to prohibit certain
unsafe operations when doing the parallel operation. If I move
PrepareParallelMode() into EnterParallelMode() then I need to pass in
contextual information to distinguish who the caller is, and possibly
extra information needed by that caller - and change the function call
for each caller, and probably update the comments for each, and in
other places, etc. etc.
I think that it just complicates things doing this. The other callers
of EnterParallelMode() are obviously currently doing their own "pre"
parallel-mode code themselves, specific to whatever parallel operation
they are doing - but nobody has thought it necessary to have to hook
this code into EnterParallelMode().
I think the "PrepareParallelMode()" name can just be changed to
something specific to plan execution, so nobody gets confused with a
name like "PrepareParallelMode()", which as you point out sounds
generic to all callers of EnterParallelMode().
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Jan 22, 2021 at 7:52 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
(1) - * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE - * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader - * backend writes into a completely new table. In the future, we can - * extend it to allow workers to write into the table. However, to allow - * parallel updates and deletes, we have to solve other problems, - * especially around combo CIDs.) + * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT + * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as + * of now, only the leader backend writes into a completely new table. InThis can read "In INSERT INTO...SELECT case, like other existing cases, only the leader backend writes into a completely new table." The reality is that workers as well as the leader can write into an empty or non-empty table in parallel, isn't it?
Sorry, I've just realized that this is in reference to the 1st patch
(v12-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch),
which implements parallel SELECT for INSERT.
In that case, data is SELECTed in parallel by the workers, but only
INSERTed by the parallel leader.
So the patch comment is, in fact, correct.
In the 3rd patch
(v12-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patch),
which implements parallel INSERT, the wording for this comment is
again altered, to reflect the fact that parallel workers also write
into the table.
Regards,
Greg Nancarrow
Fujitsu Australia
Hi,
When reading the code of rel_max_parallel_hazard_for_modify in 0001.
I thought there are so many places call table_close().
Personally, It's a little confused to me.
Do you think it's better to do the table_open/close outside of rel_max_parallel_hazard_for_modify ?
Like:
static bool rel_max_parallel_hazard_for_modify(Relation rel,
CmdType command_type,
max_parallel_hazard_context *context);
...
Relation relation = table_open(rte->relid, NoLock);
(void) rel_max_parallel_hazard_for_modify(relation, parse->commandType, &context);
table_close(relation, NoLock);
And we seems do not need the lockmode param with the above define.
Best regards,
houzj
On Mon, Jan 25, 2021 at 10:40 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi,
When reading the code of rel_max_parallel_hazard_for_modify in 0001.
I thought there are so many places call table_close().
Personally, It's a little confused to me.Do you think it's better to do the table_open/close outside of rel_max_parallel_hazard_for_modify ?
Like:
static bool rel_max_parallel_hazard_for_modify(Relation rel,
CmdType command_type,
max_parallel_hazard_context *context);
...
Relation relation = table_open(rte->relid, NoLock);
(void) rel_max_parallel_hazard_for_modify(relation, parse->commandType, &context);
table_close(relation, NoLock);And we seems do not need the lockmode param with the above define.
Yeah, the repeated cleanup at the point of return is a bit ugly.
It could be solved by changing the function to do cleanup at a common
return point, but I agree with you that in this case it could simply
be done outside the function.
Thanks, I'll make that change.
Regards,
Greg Nancarrow
Fujitsu Australia
Hi,
I have an issue of the check about column default expressions.
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
IMO, max_parallel_hazard() only check the parent table's default expressions, But if the table has partitions and its partition have its own default expressions, max_parallel_hazard() seems does not check that.
And we seems does not check that too.
I am not sure should we allow parallel insert for this case ?
Example:
-------------------------
set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
create table origin(a int);
insert into origin values(generate_series(1,5000));
create or replace function bdefault_unsafe () returns int language plpgsql parallel unsafe as $$ begin
RETURN 5;
end $$;
create table parttable1 (a int, b name) partition by range (a); create table parttable1_1 partition of parttable1 for values from (0) to (5000); create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
alter table parttable1_1 ALTER COLUMN b SET DEFAULT bdefault_unsafe();
postgres=# explain insert into parttable1 select * from origin ;
QUERY PLAN
--------------------------------------------------------------------------------
Gather (cost=0.00..41.92 rows=5865 width=0)
Workers Planned: 3
-> Insert on parttable1 (cost=0.00..41.92 rows=0 width=0)
-> Parallel Seq Scan on origin (cost=0.00..41.92 rows=1892 width=68)
(4 rows)
postgres=# explain insert into parttable1_1 select * from origin ;
QUERY PLAN
-------------------------------------------------------------------
Insert on parttable1_1 (cost=0.00..1348.00 rows=0 width=0)
-> Seq Scan on origin (cost=0.00..1348.00 rows=5000 width=68)
(2 rows)
-------------------------
Best regards,
houzj
From: Hou, Zhijie <houzj.fnst@cn.fujitsu.com>
IMO, max_parallel_hazard() only check the parent table's default expressions,
But if the table has partitions and its partition have its own default expressions,
max_parallel_hazard() seems does not check that.
And we seems does not check that too.I am not sure should we allow parallel insert for this case ?
I think we can allow parallel insert in this case, because the column value is determined according to the DEFAULT definition of the target table specified in the INSERT statement. This is described here:
https://www.postgresql.org/docs/devel/sql-createtable.html
"Defaults may be specified separately for each partition. But note that a partition's default value is not applied when inserting a tuple through a partitioned table."
So the parallel-unsafe function should not be called.
Regards
Takayuki Tsunakawa
I think we can allow parallel insert in this case, because the column value
is determined according to the DEFAULT definition of the target table
specified in the INSERT statement. This is described here:https://www.postgresql.org/docs/devel/sql-createtable.html
"Defaults may be specified separately for each partition. But note that
a partition's default value is not applied when inserting a tuple through
a partitioned table."So the parallel-unsafe function should not be called.
Thanks for the explanation.
I think you are right, I did miss it.
Best regards,
houzj
Hi,
When testing the patch with the following kind of sql.
---
Insert into part_table select 1;
Insert into part_table select generate_series(1,10000,1);
Insert into part_table select * from testfunc();
---
we usually use these sqls to initialize the table or for testing purpose.
Personally I think we do not need to do the parallel safety-check for these cases,
because there seems no chance for the select part to consider parallel.
I thought we aim to not check the safety unless parallel is possible.
, So I was thinking is it possible to avoid the check it these cases ?
I did some quick check on the code, An Immature ideal is to check if there is RTE_RELATION in query.
If no we do not check the safety-check.
I am not sure is it worth to do that, any thoughts ?
Best regards,
Houzj
On Wed, Jan 27, 2021 at 2:13 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi,
When testing the patch with the following kind of sql.
---
Insert into part_table select 1;
Insert into part_table select generate_series(1,10000,1);
Insert into part_table select * from testfunc();
---we usually use these sqls to initialize the table or for testing purpose.
Personally I think we do not need to do the parallel safety-check for these cases,
because there seems no chance for the select part to consider parallel.I thought we aim to not check the safety unless parallel is possible.
, So I was thinking is it possible to avoid the check it these cases ?I did some quick check on the code, An Immature ideal is to check if there is RTE_RELATION in query.
If no we do not check the safety-check.I am not sure is it worth to do that, any thoughts ?
Yes, I think it's worth it. It's surprising that there's not really
any optimizations for these with just the current Postgres parallel
SELECT functionality (as there's currently no way to divide the work
for these amongst the workers, even if the function/expression is
parallel-safe).
For the additional parallel-safety checks for INSERT, currently we
check that RTE_SUBQUERY is in the range-table. So I think we can
additionally check that RTE_RELATION is in the subquery range-table
(otherwise treat it as unsafe).
Regards,
Greg Nancarrow
Fujitsu Australia
Thanks for the feedback.
Posting an updated set of patches. Changes are based on feedback, as
detailed below:
[Hou]
- Deallocate list returned from RelationGetIndexList() using
list_free() after finished using list
- Regard foreign and temporary tables as parallel-restricted (instead
of parallel unsafe) for Insert
- pfree() conbin returned from TextDatumGetCString() after finished using it
- Make parallel-safety checks of index expressions more efficient,
remove some redundant function calls
- Add a few more test cases to cover certain untested parallel-safety
check cases
- Remove repeated table_close() on return, by moving table_open() &
table_close() to a higher level
- Reduce Insert parallel-safety checks required for some SQL, by
noting that the subquery must operate on a relation (check for
RTE_RELATION in subquery range-table)
[Zhihong Yu]
- Minor change to patch comment
- Wrap long line
- Remove intermediate local variable
[Tsunakawa-san]
- Update RELATION_IS_LOCAL macro to reinstate previously-removed check
on the relation being newly created in the current transaction (and so
assumed accessible only to the current backend), but for
non-parallel-mode only (since now it may be accessible to parallel
workers)
- Remove braces for one-line else
- Fix code comment
- Rename PrepareParallelMode() for plan execution, so that it's not
misinterpreted as a general function for preparation of parallel-mode
[Misc]
- Fix bug in query re-writer - hasModifyingCTE is not set in
re-written non-SELECT queries having a CTE
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v13-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v13-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 73592bc046bec90da4dc5d35d0fc25093644cc62 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 28 Jan 2021 21:06:04 +1100
Subject: [PATCH v13 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1146 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 571 ++++++++
5 files changed, 1722 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..26d9733b1a
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1146 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index e0e1ef71dd..c78f213bf8 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 081fce32e7..41ff09791d 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..55a74365c6
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,571 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
v13-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v13-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From d1de55786a24ebf3a087ed94ae9494092b4bebe2 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 28 Jan 2021 13:53:51 +1100
Subject: [PATCH v13 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a fix to the query rewriter, as it was found that for
re-written non-SELECT queries with a modifying CTE, the hasModifyingCTE flag
was not getting set to true - resulting in a test failure in the "with" tests
when force_parallel_mode=regress was in effect.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 21 +-
src/backend/optimizer/util/clauses.c | 464 +++++++++++++++++++++++++++
src/backend/rewrite/rewriteHandler.c | 70 +++-
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 1 +
7 files changed, 580 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..dd70ec2b21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index f4dd47acc7..e8414d9310 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 4e6497ff32..83ca943736 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index d2470b7c6a..02a7e05fdd 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -148,6 +158,14 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static bool rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -553,6 +571,18 @@ max_parallel_hazard(Query *parse)
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+
+ if (context.max_hazard != PROPARALLEL_UNSAFE &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ context.max_hazard = max_parallel_hazard_for_modify(parse, context.max_hazard);
+ }
return context.max_hazard;
}
@@ -786,6 +816,440 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = rel_max_parallel_hazard_for_modify(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (trigger_max_parallel_hazard_for_modify(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ ListCell *lc;
+ bool hasSubQueryOnRelation;
+ max_parallel_hazard_context context;
+ Relation rel;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * If there is no underlying query on a relation, a parallel
+ * table-modification operation is not possible (nor desirable).
+ */
+ hasSubQueryOnRelation = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ RangeTblEntry *rteSub;
+ ListCell *lcSub;
+
+ foreach(lcSub, rte->subquery->rtable)
+ {
+ rteSub = lfirst_node(RangeTblEntry, lcSub);
+ if (rteSub->rtekind == RTE_RELATION)
+ {
+ hasSubQueryOnRelation = true;
+ break;
+ }
+ }
+ if (hasSubQueryOnRelation)
+ break;
+ }
+ }
+ if (!hasSubQueryOnRelation)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == 0 ||
+ initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == 0 ?
+ PROPARALLEL_SAFE : initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase).
+ */
+ rel = table_open(rte->relid, NoLock);
+ (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType, &context);
+ table_close(rel, NoLock);
+ return context.max_hazard;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c
index 0c7508a0d8..7e8ea4d880 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -4142,6 +4142,63 @@ RewriteQuery(Query *parsetree, List *rewrite_events)
}
+/*
+ * Determine if the specified query has a modifying-CTE.
+ */
+static bool
+queryHasModifyingCTE(Query *parsetree)
+{
+ bool hasModifyingCTE = false;
+
+ if (parsetree->querySource == QSRC_ORIGINAL)
+ {
+ /* Assume original queries have hasModifyingCTE set correctly */
+ if (parsetree->hasModifyingCTE)
+ hasModifyingCTE = true;
+ }
+
+ if (!hasModifyingCTE)
+ {
+ int rt_index;
+
+ /* Recursively check subqueries */
+ rt_index = 0;
+ while (rt_index < list_length(parsetree->rtable))
+ {
+ RangeTblEntry *rte;
+ ++rt_index;
+ rte = rt_fetch(rt_index, parsetree->rtable);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasModifyingCTE = queryHasModifyingCTE(rte->subquery);
+ if (hasModifyingCTE)
+ break;
+ }
+ }
+
+ if (!hasModifyingCTE)
+ {
+ ListCell *lc;
+
+ /* Check for INSERT/UPDATE/DELETE CTEs */
+ foreach(lc, parsetree->cteList)
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ hasModifyingCTE = true;
+ break;
+ }
+ }
+ }
+ }
+
+ return hasModifyingCTE;
+}
+
+
/*
* QueryRewrite -
* Primary entry point to the query rewriter.
@@ -4207,8 +4264,9 @@ QueryRewrite(Query *parsetree)
* default tag based on the original un-rewritten query.)
*
* The Asserts verify that at most one query in the result list is marked
- * canSetTag. If we aren't checking asserts, we can fall out of the loop
- * as soon as we find the original query.
+ * canSetTag. While we're processing each query, if it's not the original
+ * query, determine if the query contains a modifying CTE and set
+ * hasModifyingCTE accordingly.
*/
origCmdType = parsetree->commandType;
foundOriginalQuery = false;
@@ -4223,17 +4281,17 @@ QueryRewrite(Query *parsetree)
Assert(query->canSetTag);
Assert(!foundOriginalQuery);
foundOriginalQuery = true;
-#ifndef USE_ASSERT_CHECKING
- break;
-#endif
}
else
{
Assert(!query->canSetTag);
- if (query->commandType == origCmdType &&
+ if (!foundOriginalQuery &&
+ query->commandType == origCmdType &&
(query->querySource == QSRC_INSTEAD_RULE ||
query->querySource == QSRC_QUAL_INSTEAD_RULE))
lastInstead = query;
+
+ query->hasModifyingCTE = queryHasModifyingCTE(query);
}
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..3cdddbfb35 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
2.27.0
v13-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v13-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 7d9aeed66b3e7d499a975a906cfa4ff90e1cab09 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 28 Jan 2021 22:03:36 +1100
Subject: [PATCH v13 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++--
src/test/regress/expected/insert_parallel.out | 120 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 20 +--
3 files changed, 145 insertions(+), 78 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..3cc029c6e3 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 26d9733b1a..3b922a2c3d 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -212,9 +212,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -251,9 +251,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -293,9 +293,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -343,9 +343,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -378,9 +378,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -414,9 +414,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -452,9 +452,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -492,9 +492,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -566,21 +566,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -609,7 +609,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -661,14 +661,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -717,7 +717,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -746,14 +746,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -810,9 +810,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -874,9 +874,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -913,7 +913,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -934,11 +934,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -979,7 +979,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1036,9 +1036,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 55a74365c6..34a191f7c5 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -250,14 +250,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -265,7 +265,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -290,7 +290,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -306,7 +306,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -314,7 +314,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -398,7 +398,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -448,7 +448,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
2.27.0
v13-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v13-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 4e40c184123c7b2c35e011867751da565fcce12c Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 27 Jan 2021 23:06:19 +1100
Subject: [PATCH v13 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 19 +-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 502 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dd70ec2b21..4d2923a2fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index e8414d9310..2d1a6af3e2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 5d90337498..bd7264bd89 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 25d4750ca6..dad91cfc3b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 83ca943736..92f75f368d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1793,7 +1797,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1841,6 +1958,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2281,96 +2399,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2389,6 +2444,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7558,7 +7620,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..41599ce44e 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,23 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing targetlist
+ * on the Gather node, in the case of an underlying ModifyTable node - currently it's
+ * not clear how to achieve this elegantly.
+ */
+ if (finalPlan != NULL && IsA(finalPlan, Gather))
+ {
+ Plan *subplan = outerPlan(finalPlan);
+
+ if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ finalPlan->targetlist = copyObject(subplan->targetlist);
+ }
+ }
+ return finalPlan;
}
/*
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index d465b9e213..dd89a63841 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3528,6 +3528,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3538,10 +3539,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3558,47 +3559,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3614,6 +3590,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index d65099c94a..543bf1255c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1170,7 +1170,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 23dec14cbd..b2ab024e69 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -264,7 +264,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
Hi Greg,
Recently, I was keeping evaluating performance of this patch(1/28 V13).
Here I find a regression test case which is parallel insert with bitmap heap scan.
when the target table has primary key or index, then the patched performance will have a 7%-19% declines than unpatched.
Could you please have a look about this?
I tried max_parallel_workers_per_gather=2/4/8, and I didn't tune other parameters(like GUCs or other enforce parallel parameters).
1. max_parallel_workers_per_gather=2(default)
target_table patched master %reg
------------------------------------------------------
without_PK_index 83.683 142.183 -41%
with_PK 382.824 321.101 19%
with_index 372.682 324.246 15%
2. max_parallel_workers_per_gather=4
target_table patched master %reg
------------------------------------------------------
without_PK_index 73.189 141.879 -48%
with_PK 362.104 329.759 10%
with_index 372.237 333.718 12%
3. max_parallel_workers_per_gather=8 (also set max_parallel_workers=16, max_worker_processes = 16)
target_table patched master %reg
------------------------------------------------------
without_PK_index 75.072 146.100 -49%
with_PK 365.312 324.339 13%
with_index 362.636 338.366 7%
Attached test_bitmap.sql which includes my test data and sql if you want to have a look.
Regards,
Tang
Attachments:
Hi,
When developing the reloption patch, I noticed some issues in the patch.
1).
- Reduce Insert parallel-safety checks required for some SQL, by noting
that the subquery must operate on a relation (check for RTE_RELATION in
subquery range-table)
+ foreach(lcSub, rte->subquery->rtable)
+ {
+ rteSub = lfirst_node(RangeTblEntry, lcSub);
+ if (rteSub->rtekind == RTE_RELATION)
+ {
+ hasSubQueryOnRelation = true;
+ break;
+ }
+ }
It seems we can not only search RTE_RELATION in rtable,
because RTE_RELATION may exist in other place like:
---
--** explain insert into target select (select * from test);
Subplan's subplan
--** with cte as (select * from test) insert into target select * from cte;
In query's ctelist.
---
May be we should use a walker function [1]static bool relation_walker(Node *node) { if (node == NULL) return false; to
search the subquery and ctelist.
2).
+--
+-- Test INSERT into temporary table with underlying query.
+-- (should not use a parallel plan)
+--
May be the comment here need some change since
we currently support parallel plan for temp table.
3)
Do you think we can add a testcase for foreign-table ?
To test parallel query with serial insert on foreign table.
[1]: static bool relation_walker(Node *node) { if (node == NULL) return false;
static bool
relation_walker(Node *node)
{
if (node == NULL)
return false;
else if (IsA(node, RangeTblEntry))
{
RangeTblEntry *rte = (RangeTblEntry *) node;
if (rte->rtekind == RTE_RELATION)
return true;
return false;
}
else if (IsA(node, Query))
{
Query *query = (Query *) node;
/* Recurse into subselects */
return query_tree_walker(query, relation_walker,
NULL, QTW_EXAMINE_RTES_BEFORE);
}
/* Recurse to check arguments */
return expression_tree_walker(node,
relation_walker,
NULL);
}
Best regards,
houzj
On Mon, Feb 1, 2021 at 8:19 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
When developing the reloption patch, I noticed some issues in the patch.
1).
- Reduce Insert parallel-safety checks required for some SQL, by noting
that the subquery must operate on a relation (check for RTE_RELATION in
subquery range-table)+ foreach(lcSub, rte->subquery->rtable) + { + rteSub = lfirst_node(RangeTblEntry, lcSub); + if (rteSub->rtekind == RTE_RELATION) + { + hasSubQueryOnRelation = true; + break; + } + } It seems we can not only search RTE_RELATION in rtable, because RTE_RELATION may exist in other place like:---
--** explain insert into target select (select * from test);
Subplan's subplan--** with cte as (select * from test) insert into target select * from cte;
In query's ctelist.
---May be we should use a walker function [1] to
search the subquery and ctelist.
Yes, the current checks are too simple, as you point out, there seem
to be more complex cases that it doesn't pick up. Unfortunately
expanding the testing for them does detract from the original
intention of this code (which was to avoid extra parallel-safety check
processing on code which can't be run in parallel). I guess the
relation walker function should additionally check for SELECT queries
only (commandType == CMD_SELECT), and exclude SELECT FOR UPDATE/SHARE
(rowMarks != NIL) too. I'll need to look further into it, but will
certainly update the code for the next version of the patch.
2).
+-- +-- Test INSERT into temporary table with underlying query. +-- (should not use a parallel plan) +--May be the comment here need some change since
we currently support parallel plan for temp table.
Thanks, it should say something like "should create the plan with
INSERT + parallel SELECT".
3)
Do you think we can add a testcase for foreign-table ?
To test parallel query with serial insert on foreign table.
I have intended to do it, but as a lower-priority task.
[1]
static bool
relation_walker(Node *node)
{
if (node == NULL)
return false;else if (IsA(node, RangeTblEntry))
{
RangeTblEntry *rte = (RangeTblEntry *) node;
if (rte->rtekind == RTE_RELATION)
return true;return false;
}else if (IsA(node, Query))
{
Query *query = (Query *) node;/* Recurse into subselects */
return query_tree_walker(query, relation_walker,
NULL, QTW_EXAMINE_RTES_BEFORE);
}/* Recurse to check arguments */
return expression_tree_walker(node,
relation_walker,
NULL);
}
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Feb 1, 2021 at 8:19 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi,
When developing the reloption patch, I noticed some issues in the patch.
1).
- Reduce Insert parallel-safety checks required for some SQL, by noting
that the subquery must operate on a relation (check for RTE_RELATION in
subquery range-table)+ foreach(lcSub, rte->subquery->rtable) + { + rteSub = lfirst_node(RangeTblEntry, lcSub); + if (rteSub->rtekind == RTE_RELATION) + { + hasSubQueryOnRelation = true; + break; + } + } It seems we can not only search RTE_RELATION in rtable, because RTE_RELATION may exist in other place like:---
--** explain insert into target select (select * from test);
Subplan's subplan--** with cte as (select * from test) insert into target select * from cte;
In query's ctelist.
---May be we should use a walker function [1] to
search the subquery and ctelist.[1]
static bool
relation_walker(Node *node)
{
if (node == NULL)
return false;else if (IsA(node, RangeTblEntry))
{
RangeTblEntry *rte = (RangeTblEntry *) node;
if (rte->rtekind == RTE_RELATION)
return true;return false;
}else if (IsA(node, Query))
{
Query *query = (Query *) node;/* Recurse into subselects */
return query_tree_walker(query, relation_walker,
NULL, QTW_EXAMINE_RTES_BEFORE);
}/* Recurse to check arguments */
return expression_tree_walker(node,
relation_walker,
NULL);
}
I've had a further look at this, and this walker function is doing a
lot of work recursing the parse tree, and I'm not sure that it
reliably retrieves the information that we;re looking for, for all
cases of different SQL queries. Unless it can be made much more
efficient and specific to our needs, I think we should not try to do
this optimization, because there's too much overhead. Also, keep in
mind that for the current parallel SELECT functionality in Postgres, I
don't see any similar optimization being attempted (and such
optimization should be attempted at the SELECT level). So I don't
think we should be attempting such optimization in this patch (but
could be attempted in a separate patch, just related to current
parallel SELECT functionality).
Regards,
Greg Nancarrow
Fujitsu Australia
I've had a further look at this, and this walker function is doing a lot
of work recursing the parse tree, and I'm not sure that it reliably retrieves
the information that we;re looking for, for all cases of different SQL
queries. Unless it can be made much more efficient and specific to our needs,
I think we should not try to do this optimization, because there's too much
overhead. Also, keep in mind that for the current parallel SELECT
functionality in Postgres, I don't see any similar optimization being
attempted (and such optimization should be attempted at the SELECT level).
So I don't think we should be attempting such optimization in this patch
(but could be attempted in a separate patch, just related to current parallel
SELECT functionality).
Yes, I agreed,
I was worried about the overhead it may bring too,
we can remove this from the current patch.
Best regards,
houzj
On Tue, Feb 2, 2021 at 7:26 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Yes, I agreed,
I was worried about the overhead it may bring too,
we can remove this from the current patch.
Posting an updated set of patches. Changes are based on feedback, as
detailed below:
[Hou]
- Corrected code that tries to check for underlying query on a
relation (but must at least check for underlying query, to rule out
VALUES). More complex tests not used, due to overhead.
- Fixed comment in tests for INSERT on temp table.
[Antonin]
- Moved Gather node targetlist fix-up code to set_plan_refs() and
updated comment (Antonin)
- Added extra tests for INSERT with RETURNING clause.
Hou: the parallel_dml patches will need slight rebasing
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v14-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v14-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From b59a22155c873bdc6bfc4986dbbfa4ba69a7aea9 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Tue, 2 Feb 2021 20:31:49 +1100
Subject: [PATCH v14 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a fix to the query rewriter, as it was found that for
re-written non-SELECT queries with a modifying CTE, the hasModifyingCTE flag
was not getting set to true - resulting in a test failure in the "with" tests
when force_parallel_mode=regress was in effect.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 21 +-
src/backend/optimizer/util/clauses.c | 452 +++++++++++++++++++++++++++
src/backend/rewrite/rewriteHandler.c | 70 ++++-
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 1 +
7 files changed, 568 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..dd70ec2b21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index f4dd47acc7..e8414d9310 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index adf68d8790..1e984461a1 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..53c43db011 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -148,6 +158,14 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static bool rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -553,6 +571,18 @@ max_parallel_hazard(Query *parse)
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+
+ if (context.max_hazard != PROPARALLEL_UNSAFE &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ context.max_hazard = max_parallel_hazard_for_modify(parse, context.max_hazard);
+ }
return context.max_hazard;
}
@@ -786,6 +816,428 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = rel_max_parallel_hazard_for_modify(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (trigger_max_parallel_hazard_for_modify(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ ListCell *lc;
+ bool hasSubQuery;
+ max_parallel_hazard_context context;
+ Relation rel;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == 0 ||
+ initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == 0 ?
+ PROPARALLEL_SAFE : initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase).
+ */
+ rel = table_open(rte->relid, NoLock);
+ (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType, &context);
+ table_close(rel, NoLock);
+ return context.max_hazard;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..02e8e88e9b 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -4160,6 +4160,63 @@ RewriteQuery(Query *parsetree, List *rewrite_events)
}
+/*
+ * Determine if the specified query has a modifying-CTE.
+ */
+static bool
+queryHasModifyingCTE(Query *parsetree)
+{
+ bool hasModifyingCTE = false;
+
+ if (parsetree->querySource == QSRC_ORIGINAL)
+ {
+ /* Assume original queries have hasModifyingCTE set correctly */
+ if (parsetree->hasModifyingCTE)
+ hasModifyingCTE = true;
+ }
+
+ if (!hasModifyingCTE)
+ {
+ int rt_index;
+
+ /* Recursively check subqueries */
+ rt_index = 0;
+ while (rt_index < list_length(parsetree->rtable))
+ {
+ RangeTblEntry *rte;
+ ++rt_index;
+ rte = rt_fetch(rt_index, parsetree->rtable);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasModifyingCTE = queryHasModifyingCTE(rte->subquery);
+ if (hasModifyingCTE)
+ break;
+ }
+ }
+
+ if (!hasModifyingCTE)
+ {
+ ListCell *lc;
+
+ /* Check for INSERT/UPDATE/DELETE CTEs */
+ foreach(lc, parsetree->cteList)
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ hasModifyingCTE = true;
+ break;
+ }
+ }
+ }
+ }
+
+ return hasModifyingCTE;
+}
+
+
/*
* QueryRewrite -
* Primary entry point to the query rewriter.
@@ -4225,8 +4282,9 @@ QueryRewrite(Query *parsetree)
* default tag based on the original un-rewritten query.)
*
* The Asserts verify that at most one query in the result list is marked
- * canSetTag. If we aren't checking asserts, we can fall out of the loop
- * as soon as we find the original query.
+ * canSetTag. While we're processing each query, if it's not the original
+ * query, determine if the query contains a modifying CTE and set
+ * hasModifyingCTE accordingly.
*/
origCmdType = parsetree->commandType;
foundOriginalQuery = false;
@@ -4241,17 +4299,17 @@ QueryRewrite(Query *parsetree)
Assert(query->canSetTag);
Assert(!foundOriginalQuery);
foundOriginalQuery = true;
-#ifndef USE_ASSERT_CHECKING
- break;
-#endif
}
else
{
Assert(!query->canSetTag);
- if (query->commandType == origCmdType &&
+ if (!foundOriginalQuery &&
+ query->commandType == origCmdType &&
(query->querySource == QSRC_INSTEAD_RULE ||
query->querySource == QSRC_QUAL_INSTEAD_RULE))
lastInstead = query;
+
+ query->hasModifyingCTE = queryHasModifyingCTE(query);
}
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..3cdddbfb35 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
2.27.0
v14-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v14-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 579ffc4af3a27b9c031b8160ddc9954f0def6a10 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 29 Jan 2021 22:35:49 +1100
Subject: [PATCH v14 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++--
src/test/regress/expected/insert_parallel.out | 126 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 20 +--
3 files changed, 148 insertions(+), 81 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..3cc029c6e3 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index ea8f465be9..72040eb955 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -173,9 +173,9 @@ create table test_data1(like test_data);
explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -341,9 +341,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -391,9 +391,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -426,9 +426,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -462,9 +462,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -500,9 +500,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -540,9 +540,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -614,21 +614,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -657,7 +657,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -710,14 +710,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -766,7 +766,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -795,14 +795,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -962,7 +962,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -983,11 +983,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1028,7 +1028,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1085,9 +1085,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index e3e651621d..0142755d9f 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -266,14 +266,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -281,7 +281,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -307,7 +307,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -323,7 +323,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -331,7 +331,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -415,7 +415,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -465,7 +465,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
2.27.0
v14-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v14-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 3a2bc79dd2e3cf635885be62d9bbaa0c19f8f6b9 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Tue, 2 Feb 2021 14:46:49 +1100
Subject: [PATCH v14 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1195 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 588 ++++++++
5 files changed, 1788 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..ea8f465be9
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1195 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index e0e1ef71dd..c78f213bf8 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 081fce32e7..41ff09791d 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..e3e651621d
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,588 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
v14-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v14-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 2ced863085f48e862a4a168a968cb8cbc0089558 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 29 Jan 2021 21:17:11 +1100
Subject: [PATCH v14 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 27 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 510 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dd70ec2b21..4d2923a2fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index e8414d9310..2d1a6af3e2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 5d90337498..bd7264bd89 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1e984461a1..6efce22a1a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1793,7 +1797,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1841,6 +1958,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2281,96 +2399,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2389,6 +2444,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7557,7 +7619,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..7968dfce9a 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1050,6 +1053,28 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, the Gather Node for INSERT was always underneath the
+ * ModifyTable node.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index d65099c94a..543bf1255c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1170,7 +1170,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
From: Amit Kapila <amit.kapila16@gmail.com>
On Mon, Jan 18, 2021 at 2:40 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:Execute EXPLAIN on Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_partselect * from test_data1;
QUERY PLAN
---------------------------------------------------------------------------
---------------------------------------------Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=44.139..44.140 rows=0 loops=1)
Buffers: shared hit=1005 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000width=8) (actual time=0.007..0.201 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=27011
Planning Time: 24.526 ms
Execution Time: 44.981 msExecute EXPLAIN on non-Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_partselect * from test_data1;
QUERY PLAN
---------------------------------------------------------------------------
---------------------------------------------Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=72.656..72.657 rows=0 loops=1)
Buffers: shared hit=22075 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000width=8) (actual time=0.010..0.175 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=72
Planning Time: 0.135 ms
Execution Time: 79.058 msSo, the results indicate that after the patch we touch more buffers
during planning which I think is because of accessing the partition
information, and during execution, the patch touches fewer buffers for
the same reason. But why this can reduce the time with patch? I think
this needs some investigation.
I guess another factor other than shared buffers is relcache and catcache. The patched version loads those cached entries for all partitions of the insert target table during the parallel-safety check in planning, while the unpatched version has to gradually build those cache entries during execution. How can wee confirm its effect?
Regards
Takayuki Tsunakawa
On Thu, Feb 4, 2021 at 6:26 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
From: Amit Kapila <amit.kapila16@gmail.com>
On Mon, Jan 18, 2021 at 2:40 PM Tang, Haiying
<tanghy.fnst@cn.fujitsu.com> wrote:Execute EXPLAIN on Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_partselect * from test_data1;
QUERY PLAN
---------------------------------------------------------------------------
---------------------------------------------Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=44.139..44.140 rows=0 loops=1)
Buffers: shared hit=1005 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000width=8) (actual time=0.007..0.201 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=27011
Planning Time: 24.526 ms
Execution Time: 44.981 msExecute EXPLAIN on non-Patched:
postgres=# explain (ANALYZE, BUFFERS, VERBOSE) insert into test_partselect * from test_data1;
QUERY PLAN
---------------------------------------------------------------------------
---------------------------------------------Insert on public.test_part (cost=0.00..15.00 rows=0 width=0) (actual
time=72.656..72.657 rows=0 loops=1)
Buffers: shared hit=22075 read=1000 dirtied=3000 written=2000
-> Seq Scan on public.test_data1 (cost=0.00..15.00 rows=1000width=8) (actual time=0.010..0.175 rows=1000 loops=1)
Output: test_data1.a, test_data1.b
Buffers: shared hit=5
Planning:
Buffers: shared hit=72
Planning Time: 0.135 ms
Execution Time: 79.058 msSo, the results indicate that after the patch we touch more buffers
during planning which I think is because of accessing the partition
information, and during execution, the patch touches fewer buffers for
the same reason. But why this can reduce the time with patch? I think
this needs some investigation.I guess another factor other than shared buffers is relcache and catcache. The patched version loads those cached entries for all partitions of the insert target table during the parallel-safety check in planning, while the unpatched version has to gradually build those cache entries during execution.
Right.
How can wee confirm its effect?
I am not sure but if your theory is correct then won't in consecutive
runs both should have the same performance?
--
With Regards,
Amit Kapila.
On Thu, Feb 4, 2021 at 11:56 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
So, the results indicate that after the patch we touch more buffers
during planning which I think is because of accessing the partition
information, and during execution, the patch touches fewer buffers for
the same reason. But why this can reduce the time with patch? I think
this needs some investigation.I guess another factor other than shared buffers is relcache and catcache. The patched version loads those cached entries for all partitions of the insert target table during the parallel-safety check in planning, while the unpatched version has to gradually build those cache entries during execution. How can wee confirm its effect?
I believe that we can confirm its effect by invalidating relcache and
catcache, in both the patched and unpatched versions, just after the
parallel-safety checks are performed in the planner, and then running
tests and comparing the performance.
So that's exactly what I did (adding a call to
InvalidateSystemCaches() just after the parallel-safety checks in the
planner).
I found that then the unpatched version always performed better than
the patched version for tests inserting 1000 records into a table with
100,200,500 and 1000 partitions.
Looking at the breakdown of the timing for each Insert, the Planning
Time was always significantly more for the patched version (expected,
because it does extra checks), but the Execution Time was very similar
for both the patched and unpatched versions.
Regards,
Greg Nancarrow
Fujitsu Australia
Hi,
I took a look into the hasModifyingCTE bugfix recently,
and found a possible bug case without the parallel insert patch.
---------------------------------
drop table if exists test_data1;
create table test_data1(a int, b int) ;
insert into test_data1 select generate_series(1,1000), generate_series(1,1000);
set force_parallel_mode=on;
CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;
SELECT * FROM bug6051;
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD select a as i from test_data1;
WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) INSERT INTO bug6051 SELECT * FROM t1;
*******
***ERROR: cannot assign XIDs during a parallel operation
*******
---------------------------------
I debugged it and it did have modifycte in the parsetree after rewrite.
I think if we can properly set the hasModifyingCTE, we can avoid this error by not consider parallel for this.
Thoughts ?
Best regards,
houzj
On Fri, Feb 5, 2021 at 2:58 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Hi,
I took a look into the hasModifyingCTE bugfix recently,
and found a possible bug case without the parallel insert patch.---------------------------------
drop table if exists test_data1;
create table test_data1(a int, b int) ;
insert into test_data1 select generate_series(1,1000), generate_series(1,1000);
set force_parallel_mode=on;CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;SELECT * FROM bug6051;
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD select a as i from test_data1;WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) INSERT INTO bug6051 SELECT * FROM t1;
*******
***ERROR: cannot assign XIDs during a parallel operation
*******
---------------------------------I debugged it and it did have modifycte in the parsetree after rewrite.
I think if we can properly set the hasModifyingCTE, we can avoid this error by not consider parallel for this.
Thanks. You've identified that the bug exists for SELECT too. I've
verified that the issue is fixed by the bugfix included in the
Parallel INSERT patch.
Are you able to review my bugfix?
Since the problem exists for SELECT in the current Postgres code, I'd
like to pull that bugfix out and provide it as a separate fix.
My concern is that there may well be a better way to fix the issue -
for example, during the re-writing, rather than after the query has
been re-written.
Regards,
Greg Nancarrow
Fujitsu Australia
I took a look into the hasModifyingCTE bugfix recently, and found a
possible bug case without the parallel insert patch.---------------------------------
drop table if exists test_data1;
create table test_data1(a int, b int) ; insert into test_data1 select
generate_series(1,1000), generate_series(1,1000); set
force_parallel_mode=on;CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;SELECT * FROM bug6051;
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD select a as
i from test_data1;WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) INSERT INTO bug6051
SELECT * FROM t1;*******
***ERROR: cannot assign XIDs during a parallel operation
*******
---------------------------------I debugged it and it did have modifycte in the parsetree after rewrite.
I think if we can properly set the hasModifyingCTE, we can avoid thiserror by not consider parallel for this.
Thanks. You've identified that the bug exists for SELECT too. I've verified
that the issue is fixed by the bugfix included in the Parallel INSERT patch.
Are you able to review my bugfix?
Since the problem exists for SELECT in the current Postgres code, I'd like
to pull that bugfix out and provide it as a separate fix.
My concern is that there may well be a better way to fix the issue - for
example, during the re-writing, rather than after the query has been
re-written.
Hi,
I took a look at the fix and have some thoughts on it.
(Please correct me if you have tried this idea and found something is wrong)
IMO, the main reason for the hasModifyingCTE=false is that:
the Rewriter did not update the hasModifyingCTE when copying the existsing 'cteList' to the rewrited one.
It seems there is only one place where ctelist will be copied separately.
-------
static Query *
rewriteRuleAction(Query *parsetree,
...
/* OK, it's safe to combine the CTE lists */
sub_action->cteList = list_concat(sub_action->cteList,
copyObject(parsetree->cteList));
+ sub_action->hasModifyingCTE |= parsetree->hasModifyingCTE;
--------
Based on the above, if we update the hasModifyingCTE here, we may solve this problem.
And there is another point here, the sub_action may be not the final parsetree.
If defined the rule like "DO INSTEAD insert into xx select xx from xx", Rewriter will
Put the ctelist into subquery in parsetree's rtable.
In this case, we may also need to update the final parsetree too.
(I think you know this case, I found same logic in the latest patch)
--------
static Query *
rewriteRuleAction(Query *parsetree,
...
if (sub_action_ptr)
+ {
*sub_action_ptr = sub_action;
+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE;
+ }
--------
And the Basic test passed.
What do you think ?
Best regards,
houzj
Hi,
While reviewing the v14 set of patches (will send my comments
shortly), I too had some reservations on how 0001 decided to go about
setting hasModifyingCTE.
On Fri, Feb 5, 2021 at 1:51 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
I took a look into the hasModifyingCTE bugfix recently, and found a
possible bug case without the parallel insert patch.---------------------------------
drop table if exists test_data1;
create table test_data1(a int, b int) ; insert into test_data1 select
generate_series(1,1000), generate_series(1,1000); set
force_parallel_mode=on;CREATE TEMP TABLE bug6051 AS
select i from generate_series(1,3) as i;SELECT * FROM bug6051;
CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD select a as
i from test_data1;WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) INSERT INTO bug6051
SELECT * FROM t1;*******
***ERROR: cannot assign XIDs during a parallel operation
*******
---------------------------------I debugged it and it did have modifycte in the parsetree after rewrite.
I think if we can properly set the hasModifyingCTE, we can avoid thiserror by not consider parallel for this.
Thanks. You've identified that the bug exists for SELECT too. I've verified
that the issue is fixed by the bugfix included in the Parallel INSERT patch.
Are you able to review my bugfix?
Since the problem exists for SELECT in the current Postgres code, I'd like
to pull that bugfix out and provide it as a separate fix.
+1, a separate patch for this seems better.
My concern is that there may well be a better way to fix the issue - for
example, during the re-writing, rather than after the query has been
re-written.Hi,
I took a look at the fix and have some thoughts on it.
(Please correct me if you have tried this idea and found something is wrong)IMO, the main reason for the hasModifyingCTE=false is that:
the Rewriter did not update the hasModifyingCTE when copying the existsing 'cteList' to the rewrited one.It seems there is only one place where ctelist will be copied separately. ------- static Query * rewriteRuleAction(Query *parsetree, ... /* OK, it's safe to combine the CTE lists */ sub_action->cteList = list_concat(sub_action->cteList, copyObject(parsetree->cteList)); + sub_action->hasModifyingCTE |= parsetree->hasModifyingCTE; --------Based on the above, if we update the hasModifyingCTE here, we may solve this problem.
And there is another point here, the sub_action may be not the final parsetree.
If defined the rule like "DO INSTEAD insert into xx select xx from xx", Rewriter will
Put the ctelist into subquery in parsetree's rtable.
In this case, we may also need to update the final parsetree too.
(I think you know this case, I found same logic in the latest patch)-------- static Query * rewriteRuleAction(Query *parsetree, ... if (sub_action_ptr) + { *sub_action_ptr = sub_action; + rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE; + } --------And the Basic test passed.
What do you think ?
That is very close to what I was going to suggest, which is this:
diff --git a/src/backend/rewrite/rewriteHandler.c
b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..3c4417af98 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -631,6 +631,8 @@ rewriteRuleAction(Query *parsetree,
checkExprHasSubLink((Node *) rule_action->returningList);
}
+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE;
+
return rule_action;
}
--
Amit Langote
EDB: http://www.enterprisedb.com
-------- static Query * rewriteRuleAction(Query *parsetree, ... if (sub_action_ptr) + { *sub_action_ptr = sub_action; + rule_action->hasModifyingCTE |=parsetree->hasModifyingCTE;
+ } --------And the Basic test passed.
What do you think ?That is very close to what I was going to suggest, which is this:
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..3c4417af98 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -631,6 +631,8 @@ rewriteRuleAction(Query *parsetree, checkExprHasSubLink((Node *) rule_action->returningList); }+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE; + return rule_action; }
if (parsetree->cteList != NIL && sub_action->commandType != CMD_UTILITY)
{
...
sub_action->cteList = list_concat(sub_action->cteList,
}
Is is possible when sub_action is CMD_UTILITY ?
In this case CTE will be copied to the newone, should we set the set the flag in this case ?
Best regard,
houzj
-------- static Query * rewriteRuleAction(Query *parsetree, ... if (sub_action_ptr) + { *sub_action_ptr = sub_action; + rule_action->hasModifyingCTE |=parsetree->hasModifyingCTE;
+ } --------And the Basic test passed.
What do you think ?That is very close to what I was going to suggest, which is this:
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..3c4417af98 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -631,6 +631,8 @@ rewriteRuleAction(Query *parsetree, checkExprHasSubLink((Node *) rule_action->returningList); }+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE; + return rule_action; }if (parsetree->cteList != NIL && sub_action->commandType !=
CMD_UTILITY)
{
...
sub_action->cteList = list_concat(sub_action->cteList,
}Is is possible when sub_action is CMD_UTILITY ?
In this case CTE will be copied to the newone, should we set the set the
flag in this case ?
Sorry , a typo in my word.
In this case CTE will not be copied to the newone, should we set the set the flag in this case ?
Best regards,
houzj
On Fri, Feb 5, 2021 at 4:25 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
That is very close to what I was going to suggest, which is this:
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..3c4417af98 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -631,6 +631,8 @@ rewriteRuleAction(Query *parsetree, checkExprHasSubLink((Node *) rule_action->returningList); }+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE; + return rule_action; }if (parsetree->cteList != NIL && sub_action->commandType !=
CMD_UTILITY)
{
...
sub_action->cteList = list_concat(sub_action->cteList,
}Is is possible when sub_action is CMD_UTILITY ?
In this case CTE will be copied to the newone, should we set the set the
flag in this case ?Sorry , a typo in my word.
In this case CTE will not be copied to the newone, should we set the set the flag in this case ?
No, strictly speaking, we probably shouldn't, because the CTE wasn't
copied in that case.
Also, I know the bitwise OR "works" in this case, but I think some
will frown on use of that for a bool.
IMHO better to use:
if (parsetree->hasModifyingCTE)
rule_action->hasModifyingCTE = true;
So patch might be something like:
diff --git a/src/backend/rewrite/rewriteHandler.c
b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..a989e02925 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -557,6 +557,8 @@ rewriteRuleAction(Query *parsetree,
/* OK, it's safe to combine the CTE lists */
sub_action->cteList = list_concat(sub_action->cteList,
copyObject(parsetree->cteList));
+ if (parsetree->hasModifyingCTE)
+ sub_action->hasModifyingCTE = true;
}
/*
@@ -594,6 +596,9 @@ rewriteRuleAction(Query *parsetree,
*sub_action_ptr = sub_action;
else
rule_action = sub_action;
+
+ if (parsetree->hasModifyingCTE)
+ sub_action->hasModifyingCTE = true;
}
/*
I'll do some further checks, because the rewriting is recursive and
tricky, so don't want to miss any cases ...
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 5, 2021 at 2:55 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 5, 2021 at 4:25 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
That is very close to what I was going to suggest, which is this:
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..3c4417af98 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -631,6 +631,8 @@ rewriteRuleAction(Query *parsetree, checkExprHasSubLink((Node *) rule_action->returningList); }+ rule_action->hasModifyingCTE |= parsetree->hasModifyingCTE; + return rule_action; }if (parsetree->cteList != NIL && sub_action->commandType !=
CMD_UTILITY)
{
...
sub_action->cteList = list_concat(sub_action->cteList,
}Is is possible when sub_action is CMD_UTILITY ?
In this case CTE will be copied to the newone, should we set the set the
flag in this case ?Sorry , a typo in my word.
In this case CTE will not be copied to the newone, should we set the set the flag in this case ?No, strictly speaking, we probably shouldn't, because the CTE wasn't
copied in that case.
Right.
Also, I know the bitwise OR "works" in this case, but I think some
will frown on use of that for a bool.
IMHO better to use:if (parsetree->hasModifyingCTE)
rule_action->hasModifyingCTE = true;So patch might be something like:
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..a989e02925 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -557,6 +557,8 @@ rewriteRuleAction(Query *parsetree, /* OK, it's safe to combine the CTE lists */ sub_action->cteList = list_concat(sub_action->cteList, copyObject(parsetree->cteList)); + if (parsetree->hasModifyingCTE) + sub_action->hasModifyingCTE = true; }/* @@ -594,6 +596,9 @@ rewriteRuleAction(Query *parsetree, *sub_action_ptr = sub_action; else rule_action = sub_action; + + if (parsetree->hasModifyingCTE) + sub_action->hasModifyingCTE = true; }
That may be better.
BTW, the original query's cteList is copied into sub_action query but
not into rule_action for reasons I haven't looked very closely into,
even though we'd like to ultimately set the latter's hasModifyingCTE
to reflect the original query's, right? So we should do the following
at some point before returning:
if (sub_action->hasModifyingCTE)
rule_action->hasModifyingCTE = true;
I'll do some further checks, because the rewriting is recursive and
tricky, so don't want to miss any cases ...
Always a good idea.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Fri, Feb 5, 2021 at 5:21 PM Amit Langote <amitlangote09@gmail.com> wrote:
BTW, the original query's cteList is copied into sub_action query but
not into rule_action for reasons I haven't looked very closely into,
even though we'd like to ultimately set the latter's hasModifyingCTE
to reflect the original query's, right? So we should do the following
at some point before returning:if (sub_action->hasModifyingCTE)
rule_action->hasModifyingCTE = true;
Actually, rule_action will usually point to sub_action (in which case,
no need to copy to rule_action), except if the rule action is an
INSERT...SELECT, which seems to be handled by some "kludge" according
to the following comment (and KLUDGE ALERT comment in the function
that is called):
/*
* Adjust rule action and qual to offset its varnos, so that we can merge
* its rtable with the main parsetree's rtable.
*
* If the rule action is an INSERT...SELECT, the OLD/NEW rtable entries
* will be in the SELECT part, and we have to modify that rather than the
* top-level INSERT (kluge!).
*/
sub_action = getInsertSelectQuery(rule_action, &sub_action_ptr);
So in that case (sub_action_ptr != NULL), within rule_action there is
a pointer to sub_action (RTE for the subquery), so whenever sub_action
is re-created, this pointer needs to be fixed-up.
It looks like I might need to copy hasModifyingCTE back to rule_action
in this case - but not 100% sure on it yet - still checking that. All
tests run so far pass without doing that though.
This is one reason for my original approach (though I admit, it was
not optimal) because at least it was reliable and detected the
modifyingCTE after all the rewriting and kludgy code had finished.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 5, 2021 at 4:53 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 5, 2021 at 5:21 PM Amit Langote <amitlangote09@gmail.com> wrote:
BTW, the original query's cteList is copied into sub_action query but
not into rule_action for reasons I haven't looked very closely into,
even though we'd like to ultimately set the latter's hasModifyingCTE
to reflect the original query's, right? So we should do the following
at some point before returning:if (sub_action->hasModifyingCTE)
rule_action->hasModifyingCTE = true;Actually, rule_action will usually point to sub_action (in which case,
no need to copy to rule_action), except if the rule action is an
INSERT...SELECT, which seems to be handled by some "kludge" according
to the following comment (and KLUDGE ALERT comment in the function
that is called):/*
* Adjust rule action and qual to offset its varnos, so that we can merge
* its rtable with the main parsetree's rtable.
*
* If the rule action is an INSERT...SELECT, the OLD/NEW rtable entries
* will be in the SELECT part, and we have to modify that rather than the
* top-level INSERT (kluge!).
*/
sub_action = getInsertSelectQuery(rule_action, &sub_action_ptr);So in that case (sub_action_ptr != NULL), within rule_action there is
a pointer to sub_action (RTE for the subquery), so whenever sub_action
is re-created, this pointer needs to be fixed-up.
It looks like I might need to copy hasModifyingCTE back to rule_action
in this case - but not 100% sure on it yet - still checking that. All
tests run so far pass without doing that though.
I guess we just don't have a test case where the rule_action query is
actually parallelized, like one that houzj shared a few emails ago.
This is one reason for my original approach (though I admit, it was
not optimal) because at least it was reliable and detected the
modifyingCTE after all the rewriting and kludgy code had finished.
Yeah it's hard to go through all of this highly recursive legacy code
to be sure that hasModifyingCTE is consistent with reality in *all*
cases, but let's try to do it. No other has* flags are set
after-the-fact, so I wouldn't bet on a committer letting this one
through.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Fri, Feb 5, 2021 at 8:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
This is one reason for my original approach (though I admit, it was
not optimal) because at least it was reliable and detected the
modifyingCTE after all the rewriting and kludgy code had finished.Yeah it's hard to go through all of this highly recursive legacy code
to be sure that hasModifyingCTE is consistent with reality in *all*
cases, but let's try to do it. No other has* flags are set
after-the-fact, so I wouldn't bet on a committer letting this one
through.
I have debugged the code a bit more now, and the following patch seems
to correctly fix the issue, at least for the known test cases.
(i.e. SELECT case, shared by houzj, and the INSERT...SELECT case, as
in the "with" regression tests, for which I originally detected the
issue)
diff --git a/src/backend/rewrite/rewriteHandler.c
b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..8f695b32ec 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -557,6 +557,12 @@ rewriteRuleAction(Query *parsetree,
/* OK, it's safe to combine the CTE lists */
sub_action->cteList = list_concat(sub_action->cteList,
copyObject(parsetree->cteList));
+ if (parsetree->hasModifyingCTE)
+ {
+ sub_action->hasModifyingCTE = true;
+ if (sub_action_ptr)
+ rule_action->hasModifyingCTE = true;
+ }
}
/*
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 5, 2021 at 6:56 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 5, 2021 at 8:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
This is one reason for my original approach (though I admit, it was
not optimal) because at least it was reliable and detected the
modifyingCTE after all the rewriting and kludgy code had finished.Yeah it's hard to go through all of this highly recursive legacy code
to be sure that hasModifyingCTE is consistent with reality in *all*
cases, but let's try to do it. No other has* flags are set
after-the-fact, so I wouldn't bet on a committer letting this one
through.I have debugged the code a bit more now, and the following patch seems
to correctly fix the issue, at least for the known test cases.
(i.e. SELECT case, shared by houzj, and the INSERT...SELECT case, as
in the "with" regression tests, for which I originally detected the
issue)diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..8f695b32ec 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -557,6 +557,12 @@ rewriteRuleAction(Query *parsetree, /* OK, it's safe to combine the CTE lists */ sub_action->cteList = list_concat(sub_action->cteList, copyObject(parsetree->cteList)); + if (parsetree->hasModifyingCTE) + { + sub_action->hasModifyingCTE = true; + if (sub_action_ptr) + rule_action->hasModifyingCTE = true; + } }
That seems good enough as far as I am concerned. Although either an
Assert as follows or a comment why the if (sub_action_ptr) is needed
seems warranted.
if (sub_action_ptr)
rule_action->hasModifyingCTE = true;
else
Assert(sub_action == rule_action);
Does the Assert seem overly confident?
--
Amit Langote
EDB: http://www.enterprisedb.com
On Fri, Feb 5, 2021 at 11:12 PM Amit Langote <amitlangote09@gmail.com> wrote:
That seems good enough as far as I am concerned. Although either an
Assert as follows or a comment why the if (sub_action_ptr) is needed
seems warranted.if (sub_action_ptr)
rule_action->hasModifyingCTE = true;
else
Assert(sub_action == rule_action);Does the Assert seem overly confident?
No, the Assert is exactly right, and I'll add a comment too.
See below.
I'll post the patch separately, if you can't see any further issues.
diff --git a/src/backend/rewrite/rewriteHandler.c
b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..05b80bd347 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -557,6 +557,21 @@ rewriteRuleAction(Query *parsetree,
/* OK, it's safe to combine the CTE lists */
sub_action->cteList = list_concat(sub_action->cteList,
copyObject(parsetree->cteList));
+
+ /*
+ * If the hasModifyingCTE flag is set in the source parsetree from
+ * which the CTE list is copied, the flag needs to be set in the
+ * sub_action and, if applicable, in the rule_action (INSERT...SELECT
+ * case).
+ */
+ if (parsetree->hasModifyingCTE)
+ {
+ sub_action->hasModifyingCTE = true;
+ if (sub_action_ptr)
+ rule_action->hasModifyingCTE = true;
+ else
+ Assert(sub_action == rule_action);
+ }
}
/*
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 5, 2021 at 11:01 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 5, 2021 at 11:12 PM Amit Langote <amitlangote09@gmail.com> wrote:
That seems good enough as far as I am concerned. Although either an
Assert as follows or a comment why the if (sub_action_ptr) is needed
seems warranted.if (sub_action_ptr)
rule_action->hasModifyingCTE = true;
else
Assert(sub_action == rule_action);Does the Assert seem overly confident?
No, the Assert is exactly right, and I'll add a comment too.
See below.
I'll post the patch separately, if you can't see any further issues.diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c index 0672f497c6..05b80bd347 100644 --- a/src/backend/rewrite/rewriteHandler.c +++ b/src/backend/rewrite/rewriteHandler.c @@ -557,6 +557,21 @@ rewriteRuleAction(Query *parsetree, /* OK, it's safe to combine the CTE lists */ sub_action->cteList = list_concat(sub_action->cteList, copyObject(parsetree->cteList)); + + /* + * If the hasModifyingCTE flag is set in the source parsetree from + * which the CTE list is copied, the flag needs to be set in the + * sub_action and, if applicable, in the rule_action (INSERT...SELECT + * case). + */ + if (parsetree->hasModifyingCTE) + { + sub_action->hasModifyingCTE = true; + if (sub_action_ptr) + rule_action->hasModifyingCTE = true; + else + Assert(sub_action == rule_action); + } }
LGTM, thank you.
--
Amit Langote
EDB: http://www.enterprisedb.com
Posting an updated set of patches.
The only update is to include an improved, but only temporary, fix to
the query rewriter hasModifyingCTE issue (I separately posted a patch
for this but Tom Lane concluded that the issue is more complex than
initially thought, and no easy fix could be confidently attempted at
this point in time, so nothing ended up getting pushed - the issue
needs further investigation at another time).
See also comment in patch 0001.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v15-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v15-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From df7fd8fe90542a0b1efbedeb2f949b2979964831 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Mon, 8 Feb 2021 17:41:47 +1100
Subject: [PATCH v15 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a TEMPORARY fix to the query rewriter.
Discussion: https://www.postgresql-archive.org/Bug-in-query-rewriter-hasModifyingCTE-not-getting-set-td6176917.html
It was found that for re-written queries with a modifying CTE, the
hasModifyingCTE flag was not getting set to true - resulting in a test failure
in the "with" tests when force_parallel_mode=regress was in effect, as it
allowed a CTE with a DELETE statement to be executed in parallel-mode.
This TEMPORARY fix does not fully and properly address the issue, so the problem
needs to be revisited.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 21 +-
src/backend/optimizer/util/clauses.c | 452 +++++++++++++++++++++++++++
src/backend/rewrite/rewriteHandler.c | 22 ++
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 1 +
7 files changed, 526 insertions(+), 10 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..dd70ec2b21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index f4dd47acc7..e8414d9310 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index adf68d8790..1e984461a1 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,7 +334,8 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(parse->commandType)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..53c43db011 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -148,6 +158,14 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context);
+static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
+static bool rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
/*****************************************************************************
* Aggregate-function clause manipulation
@@ -553,6 +571,18 @@ max_parallel_hazard(Query *parse)
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
+ /*
+ * Additional parallel-mode safety checks are required in order to
+ * allow an underlying parallel query to be used for a
+ * table-modification command that is supported in parallel-mode.
+ */
+
+ if (context.max_hazard != PROPARALLEL_UNSAFE &&
+ IsModifySupportedInParallelMode(parse->commandType))
+ {
+ context.max_hazard = max_parallel_hazard_for_modify(parse, context.max_hazard);
+ }
return context.max_hazard;
}
@@ -786,6 +816,428 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * trigger_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * index_expr_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+index_expr_max_parallel_hazard_for_modify(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * domain_max_parallel_hazard_for_modify
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * rel_max_parallel_hazard_for_modify
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = rel_max_parallel_hazard_for_modify(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (index_expr_max_parallel_hazard_for_modify(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (trigger_max_parallel_hazard_for_modify(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (domain_max_parallel_hazard_for_modify(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * max_parallel_hazard_for_modify
+ *
+ * Determines the worst parallel-mode hazard level for the specified
+ * table-modification statement, based on the statement attributes and
+ * target table. An initial max parallel hazard level may optionally be
+ * supplied. The search returns the earliest in the following list:
+ * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ */
+char
+max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard)
+{
+ RangeTblEntry *rte;
+ ListCell *lc;
+ bool hasSubQuery;
+ max_parallel_hazard_context context;
+ Relation rel;
+
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
+
+ /*
+ * Setup the context used in finding the max parallel-mode hazard.
+ */
+ Assert(initial_max_parallel_hazard == 0 ||
+ initial_max_parallel_hazard == PROPARALLEL_SAFE ||
+ initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
+ context.max_hazard = initial_max_parallel_hazard == 0 ?
+ PROPARALLEL_SAFE : initial_max_parallel_hazard;
+ context.max_interesting = PROPARALLEL_UNSAFE;
+ context.safe_param_ids = NIL;
+
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase).
+ */
+ rel = table_open(rte->relid, NoLock);
+ (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType, &context);
+ table_close(rel, NoLock);
+ return context.max_hazard;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/rewrite/rewriteHandler.c b/src/backend/rewrite/rewriteHandler.c
index 0672f497c6..28c83ec999 100644
--- a/src/backend/rewrite/rewriteHandler.c
+++ b/src/backend/rewrite/rewriteHandler.c
@@ -557,6 +557,28 @@ rewriteRuleAction(Query *parsetree,
/* OK, it's safe to combine the CTE lists */
sub_action->cteList = list_concat(sub_action->cteList,
copyObject(parsetree->cteList));
+
+ /*
+ * WARNING: The code below is just a TEMPORARY fix to the issue of
+ * the query rewriter not maintaining the hasModifyingCTE flag.
+ * The issue needs to be revisited and a proper fix developed.
+ *
+ * See discussion: https://www.postgresql-archive.org/Bug-in-query
+ * -rewriter-hasModifyingCTE-not-getting-set-td6176917.html
+ *
+ * If the hasModifyingCTE flag is set in the source parsetree from
+ * which the CTE list is copied, the flag needs to be set in the
+ * sub_action and, if applicable, in the rule_action (INSERT...SELECT
+ * case).
+ */
+ if (parsetree->hasModifyingCTE)
+ {
+ sub_action->hasModifyingCTE = true;
+ if (sub_action_ptr)
+ rule_action->hasModifyingCTE = true;
+ else
+ Assert(sub_action == rule_action);
+ }
}
/*
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..3cdddbfb35 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
#endif /* CLAUSES_H */
--
2.27.0
v15-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v15-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 19828077c4240e7e3495a846e92dad37e0326973 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 29 Jan 2021 22:35:49 +1100
Subject: [PATCH v15 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 ++++++++++--
src/test/regress/expected/insert_parallel.out | 126 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 20 +--
3 files changed, 148 insertions(+), 81 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..3cc029c6e3 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index ea8f465be9..72040eb955 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -173,9 +173,9 @@ create table test_data1(like test_data);
explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -341,9 +341,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -391,9 +391,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -426,9 +426,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -462,9 +462,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -500,9 +500,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -540,9 +540,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -614,21 +614,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -657,7 +657,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -710,14 +710,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -766,7 +766,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -795,14 +795,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -962,7 +962,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -983,11 +983,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1028,7 +1028,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1085,9 +1085,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index e3e651621d..0142755d9f 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -266,14 +266,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -281,7 +281,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -307,7 +307,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -323,7 +323,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -331,7 +331,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -415,7 +415,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -465,7 +465,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
--
2.27.0
v15-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v15-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From b9d275b8f1f86702d3f8dcafe3bb678434a4cc1b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 29 Jan 2021 21:17:11 +1100
Subject: [PATCH v15 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 27 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 510 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dd70ec2b21..4d2923a2fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index e8414d9310..2d1a6af3e2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 5d90337498..bd7264bd89 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1e984461a1..6efce22a1a 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1793,7 +1797,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1841,6 +1958,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2281,96 +2399,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2389,6 +2444,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7557,7 +7619,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..7968dfce9a 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1050,6 +1053,28 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, the Gather Node for INSERT was always underneath the
+ * ModifyTable node.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index d65099c94a..543bf1255c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1170,7 +1170,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v15-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v15-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 18621f87de53f7ff79c1d696bc18972c96ae997d Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Tue, 2 Feb 2021 14:46:49 +1100
Subject: [PATCH v15 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1195 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 588 ++++++++
5 files changed, 1788 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..ea8f465be9
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1195 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 12bb67e491..37dc86359b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 59b416fd80..4504af73cb 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..e3e651621d
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,588 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
Posting an updated set of patches.
A minor comment about doc.
+ <para>
+ Where the above target table features are determined to be, at worst,
+ parallel-restricted, rather than parallel-unsafe, at least a parallel table
+ scan may be used in the query plan for the <literal>INSERT</literal>
+ statement. For more information about Parallel Safety, see
+ <xref linkend="parallel-safety"/>.
+ </para>
It seems does not mention that if target table is a foreign/temp table, a parallel table scan may be used.
So how about:
+ <para>
+ Where the target table is a foreign/temporary table or the above target table features
+ are determined to be, at worst, parallel-restricted, rather than parallel-unsafe,
+ at least a parallel table scan may be used in the query plan for the
+ <literal>INSERT</literal> statement. For more information about Parallel Safety,
+ see <xref linkend="parallel-safety"/>.
+ </para>
Best regards,
houzj
On Mon, Feb 1, 2021 at 7:20 PM Tang, Haiying <tanghy.fnst@cn.fujitsu.com> wrote:
Hi Greg,
Recently, I was keeping evaluating performance of this patch(1/28 V13).
Here I find a regression test case which is parallel insert with bitmap heap scan.
when the target table has primary key or index, then the patched performance will have a 7%-19% declines than unpatched.Could you please have a look about this?
I tried max_parallel_workers_per_gather=2/4/8, and I didn't tune other parameters(like GUCs or other enforce parallel parameters).
1. max_parallel_workers_per_gather=2(default)
target_table patched master %reg
------------------------------------------------------
without_PK_index 83.683 142.183 -41%
with_PK 382.824 321.101 19%
with_index 372.682 324.246 15%2. max_parallel_workers_per_gather=4
target_table patched master %reg
------------------------------------------------------
without_PK_index 73.189 141.879 -48%
with_PK 362.104 329.759 10%
with_index 372.237 333.718 12%3. max_parallel_workers_per_gather=8 (also set max_parallel_workers=16, max_worker_processes = 16)
target_table patched master %reg
------------------------------------------------------
without_PK_index 75.072 146.100 -49%
with_PK 365.312 324.339 13%
with_index 362.636 338.366 7%Attached test_bitmap.sql which includes my test data and sql if you want to have a look.
Hi,
Did it actually use a parallel plan in your testing?
When I ran these tests with the Parallel INSERT patch applied, it did
not naturally choose a parallel plan for any of these cases.
So we can hardly blame the parallel insert with bitmap heap scan for
having worse performance, when based on costings, it doesn't actually
choose to use a parallel plan in this case.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Feb 8, 2021 at 6:00 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Posting an updated set of patches.
A minor comment about doc.
+ <para> + Where the above target table features are determined to be, at worst, + parallel-restricted, rather than parallel-unsafe, at least a parallel table + scan may be used in the query plan for the <literal>INSERT</literal> + statement. For more information about Parallel Safety, see + <xref linkend="parallel-safety"/>. + </para>It seems does not mention that if target table is a foreign/temp table, a parallel table scan may be used.
So how about:
+ <para> + Where the target table is a foreign/temporary table or the above target table features + are determined to be, at worst, parallel-restricted, rather than parallel-unsafe, + at least a parallel table scan may be used in the query plan for the + <literal>INSERT</literal> statement. For more information about Parallel Safety, + see <xref linkend="parallel-safety"/>. + </para>
Thanks. You're right, I should probably update the docs to clarify
those two cases.
(I had removed them from the list of parallel-unsafe things, but not
pointed out that a parallel table scan could still be used in these
cases).
Regards,
Greg Nancarrow
Fujitsu Australia
Did it actually use a parallel plan in your testing?
When I ran these tests with the Parallel INSERT patch applied, it did
not naturally choose a parallel plan for any of these cases.
Yes, these cases pick parallel plan naturally on my test environment.
postgres=# explain verbose insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
---------------------------------------------------------------------------------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0 width=0)
-> Parallel Bitmap Heap Scan on public.x1 (cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND (x1.a > 199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808 width=0)
-> Bitmap Index Scan on x1_a_idx (cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx (cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)
PSA is my postgresql.conf file, maybe you can have a look. Besides, I didn't do any parameters tuning in my test session.
Regards,
Tang
Attachments:
Did it actually use a parallel plan in your testing?
When I ran these tests with the Parallel INSERT patch applied, it did
not naturally choose a parallel plan for any of these cases.Yes, these cases pick parallel plan naturally on my test environment.
postgres=# explain verbose insert into testscan select a from x where
a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
----------------------------------------------------------------------
-----------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0
width=0)
-> Parallel Bitmap Heap Scan on public.x1
(cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND (x1.a >
199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808
width=0)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)PSA is my postgresql.conf file, maybe you can have a look. Besides, I didn't
do any parameters tuning in my test session.
I reproduced this on my machine.
I think we'd better do "analyze" before insert which helps reproduce this easier.
Like:
-----
analyze;
explain analyze verbose insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
-----
Best regards,
houzj
Greg, all
Thanks a lot for your work on this.
On Mon, Feb 8, 2021 at 3:53 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches.
I've been looking at these patches, initially with an intention to
review mainly any partitioning-related concerns, but have some general
thoughts as well concerning mostly the patches 0001 and 0002.
* I've seen review comments on this thread where I think it's been
suggested that whatever max_parallel_hazard_for_modify() does had
better have been integrated into max_parallel_hazard() such that
there's no particular need for that function to exist. For example,
the following:
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->onConflict != NULL && parse->onConflict->action ==
ONCONFLICT_UPDATE)
+ return PROPARALLEL_UNSAFE;
could be placed in the following block in max_parallel_hazard():
/*
* When we're first invoked on a completely unplanned tree, we must
* recurse into subqueries so to as to locate parallel-unsafe constructs
* anywhere in the tree.
*/
else if (IsA(node, Query))
{
Query *query = (Query *) node;
/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}
Furthermore, the following:
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase).
+ */
+ rel = table_open(rte->relid, NoLock);
+ (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType,
&context);
+ table_close(rel, NoLock);
can itself be wrapped in a function that's called from
max_parallel_hazard() by adding a new block for RangeTblEntry nodes
and passing QTW_EXAMINE_RTES_BEFORE to query_tree_walker().
That brings me to to this part of the hunk:
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
The justification for this given in:
/messages/by-id/CAJcOf-dF9ohqub_D805k57Y_AuDLeAQfvtaax9SpwjTSEVdiXg@mail.gmail.com
seems to be that the failure of a test case in
partition-concurrent-attach isolation suite is prevented if finding no
subquery RTEs in the query is flagged as parallel unsafe, which in
turn stops max_parallel_hazard_modify() from locking partitions for
safety checks in such cases. But it feels unprincipled to have this
code to work around a specific test case that's failing. I'd rather
edit the failing test case to disable parallel execution as
Tsunakawa-san suggested.
* Regarding function names:
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+
max_parallel_hazard_context *context);
+static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
+
max_parallel_hazard_context *context);
+static bool domain_max_parallel_hazard_for_modify(Oid typid,
max_parallel_hazard_context *context);
+static bool rel_max_parallel_hazard_for_modify(Relation rel,
+ CmdType command_type,
+
max_parallel_hazard_context *context)
IMO, it would be better to name these
target_rel_trigger_max_parallel_hazard(),
target_rel_index_max_parallel_hazard(), etc. rather than have
_for_modify at the end of these names to better connote that they
check the parallel safety of applying the modify operation to a given
target relation. Also, put these prototypes just below that of
max_parallel_hazard() to have related things close by.
Attached please see v15_delta.diff showing the changes suggested above.
* I suspect that the following is broken in light of concurrent
attachment of partitions.
+
+ /* Recursively check each partition ... */
+ pdesc = RelationGetPartitionDesc(rel);
I think we'd need to use CreatePartitionDirectory() and retrieve the
PartitionDesc using PartitionDirectoryLookup(). Something we already
do when opening partitions for SELECT planning.
* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
prepare q as insert into rp select * from foo where a%2 = 0;
explain execute q;
QUERY PLAN
-------------------------------------------------------------------------------
Gather (cost=1000.00..13041.54 rows=5642 width=0)
Workers Planned: 2
-> Insert on rp (cost=0.00..11477.34 rows=0 width=0)
-> Parallel Seq Scan on foo (cost=0.00..11477.34 rows=2351 width=4)
Filter: ((a % 2) = 0)
(5 rows)
-- create a parallel unsafe trigger (that's actually marked so)
directly on a partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();CREATE TRIGGER
-- plan still parallel
explain execute q;
QUERY PLAN
-------------------------------------------------------------------------------
Gather (cost=1000.00..13041.54 rows=5642 width=0)
Workers Planned: 2
-> Insert on rp (cost=0.00..11477.34 rows=0 width=0)
-> Parallel Seq Scan on foo (cost=0.00..11477.34 rows=2351 width=4)
Filter: ((a % 2) = 0)
(5 rows)
-- and because it is
execute q;
ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement
-- OTOH, altering parent correctly discards the parallel plan
create trigger ai_rp after insert on rp for each row execute function
make_table();
explain execute q;
QUERY PLAN
----------------------------------------------------------------
Insert on rp (cost=0.00..19425.00 rows=0 width=0)
-> Seq Scan on foo (cost=0.00..19425.00 rows=5000 width=4)
Filter: ((a % 2) = 0)
(3 rows)
It's fair to argue that it would rarely make sense to use PREPARE for
bulk loads, but we need to tighten things up a bit here regardless.
--
Amit Langote
EDB: http://www.enterprisedb.com
Attachments:
v15_delta.diffapplication/octet-stream; name=v15_delta.diffDownload
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 53c43db011..ab0fa77ec4 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -98,6 +98,8 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *targetRTE; /* query's target relation if any */
+ CmdType commandType;
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -108,6 +110,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -158,15 +169,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
- max_parallel_hazard_context *context);
-static bool index_expr_max_parallel_hazard_for_modify(Relation rel,
- max_parallel_hazard_context *context);
-static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context);
-static bool rel_max_parallel_hazard_for_modify(Relation rel,
- CmdType command_type,
- max_parallel_hazard_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -570,19 +572,11 @@ max_parallel_hazard(Query *parse)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.targetRTE = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.commandType = parse->commandType;
(void) max_parallel_hazard_walker((Node *) parse, &context);
- /*
- * Additional parallel-mode safety checks are required in order to
- * allow an underlying parallel query to be used for a
- * table-modification command that is supported in parallel-mode.
- */
-
- if (context.max_hazard != PROPARALLEL_UNSAFE &&
- IsModifySupportedInParallelMode(parse->commandType))
- {
- context.max_hazard = max_parallel_hazard_for_modify(parse, context.max_hazard);
- }
return context.max_hazard;
}
@@ -787,6 +781,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->targetRTE == NULL)
+ return false;
+
+ if (rte == context->targetRTE)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -804,10 +811,28 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (query->onConflict != NULL &&
+ query->onConflict->action == ONCONFLICT_UPDATE)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->targetRTE != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -817,12 +842,12 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
/*
- * trigger_max_parallel_hazard_for_modify
+ * target_rel_trigger_max_parallel_hazard
*
* Finds the maximum parallel-mode hazard level for the specified trigger data.
*/
static bool
-trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
max_parallel_hazard_context *context)
{
int i;
@@ -853,13 +878,13 @@ trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc,
}
/*
- * index_expr_max_parallel_hazard_for_modify
+ * target_rel_index_expr_max_parallel_hazard
*
* Finds the maximum parallel-mode hazard level for any existing index
* expressions of a specified relation.
*/
static bool
-index_expr_max_parallel_hazard_for_modify(Relation rel,
+target_rel_index_expr_max_parallel_hazard(Relation rel,
max_parallel_hazard_context *context)
{
List *index_oid_list;
@@ -924,7 +949,7 @@ index_expr_max_parallel_hazard_for_modify(Relation rel,
}
/*
- * domain_max_parallel_hazard_for_modify
+ * target_rel_domain_max_parallel_hazard
*
* Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
* Only any CHECK expressions are examined for parallel safety.
@@ -934,7 +959,7 @@ index_expr_max_parallel_hazard_for_modify(Relation rel,
*
*/
static bool
-domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context)
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
{
Relation con_rel;
ScanKeyData key[1];
@@ -994,15 +1019,31 @@ domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *co
}
/*
- * rel_max_parallel_hazard_for_modify
+ * target_rel_max_parallel_hazard
*
* Determines the maximum parallel-mode hazard level for modification
* of a specified relation.
*/
static bool
-rel_max_parallel_hazard_for_modify(Relation rel,
- CmdType command_type,
- max_parallel_hazard_context *context)
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ Relation targetRel = table_open(context->targetRTE->relid,
+ context->targetRTE->rellockmode);
+ bool max_hazard_found;
+
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->commandType,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
{
TupleDesc tupdesc;
int attnum;
@@ -1076,9 +1117,9 @@ rel_max_parallel_hazard_for_modify(Relation rel,
Relation part_rel;
part_rel = table_open(pdesc->oids[i], AccessShareLock);
- max_hazard_found = rel_max_parallel_hazard_for_modify(part_rel,
- command_type,
- context);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
table_close(part_rel, AccessShareLock);
if (max_hazard_found)
{
@@ -1091,7 +1132,7 @@ rel_max_parallel_hazard_for_modify(Relation rel,
* If there are any index expressions, check that they are parallel-mode
* safe.
*/
- if (index_expr_max_parallel_hazard_for_modify(rel, context))
+ if (target_rel_index_expr_max_parallel_hazard(rel, context))
{
return true;
}
@@ -1101,7 +1142,7 @@ rel_max_parallel_hazard_for_modify(Relation rel,
*/
if (rel->trigdesc != NULL)
{
- if (trigger_max_parallel_hazard_for_modify(rel->trigdesc, context))
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
{
return true;
}
@@ -1136,7 +1177,7 @@ rel_max_parallel_hazard_for_modify(Relation rel,
*/
if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
{
- if (domain_max_parallel_hazard_for_modify(att->atttypid, context))
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
{
return true;
}
@@ -1168,77 +1209,6 @@ rel_max_parallel_hazard_for_modify(Relation rel,
return false;
}
-/*
- * max_parallel_hazard_for_modify
- *
- * Determines the worst parallel-mode hazard level for the specified
- * table-modification statement, based on the statement attributes and
- * target table. An initial max parallel hazard level may optionally be
- * supplied. The search returns the earliest in the following list:
- * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
- */
-char
-max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard)
-{
- RangeTblEntry *rte;
- ListCell *lc;
- bool hasSubQuery;
- max_parallel_hazard_context context;
- Relation rel;
-
-
- /*
- * UPDATE is not currently supported in parallel-mode, so prohibit
- * INSERT...ON CONFLICT...DO UPDATE...
- * In order to support update, even if only in the leader, some
- * further work would need to be done. A mechanism would be needed
- * for sharing combo-cids between leader and workers during
- * parallel-mode, since for example, the leader might generate a
- * combo-cid and it needs to be propagated to the workers.
- */
- if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE)
- return PROPARALLEL_UNSAFE;
-
- /*
- * If there is no underlying SELECT, a parallel table-modification
- * operation is not possible (nor desirable).
- */
- hasSubQuery = false;
- foreach(lc, parse->rtable)
- {
- rte = lfirst_node(RangeTblEntry, lc);
- if (rte->rtekind == RTE_SUBQUERY)
- {
- hasSubQuery = true;
- break;
- }
- }
- if (!hasSubQuery)
- return PROPARALLEL_UNSAFE;
-
- /*
- * Setup the context used in finding the max parallel-mode hazard.
- */
- Assert(initial_max_parallel_hazard == 0 ||
- initial_max_parallel_hazard == PROPARALLEL_SAFE ||
- initial_max_parallel_hazard == PROPARALLEL_RESTRICTED);
- context.max_hazard = initial_max_parallel_hazard == 0 ?
- PROPARALLEL_SAFE : initial_max_parallel_hazard;
- context.max_interesting = PROPARALLEL_UNSAFE;
- context.safe_param_ids = NIL;
-
- rte = rt_fetch(parse->resultRelation, parse->rtable);
-
- /*
- * The target table is already locked by the caller (this is done in the
- * parse/analyze phase).
- */
- rel = table_open(rte->relid, NoLock);
- (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType, &context);
- table_close(rel, NoLock);
- return context.max_hazard;
-}
-
/*****************************************************************************
* Check clauses for nonstrict functions
*****************************************************************************/
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
Greg, all
Thanks a lot for your work on this.
On Mon, Feb 8, 2021 at 3:53 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches.
I've been looking at these patches, initially with an intention to
review mainly any partitioning-related concerns, but have some general
thoughts as well concerning mostly the patches 0001 and 0002.* I've seen review comments on this thread where I think it's been
suggested that whatever max_parallel_hazard_for_modify() does had
better have been integrated into max_parallel_hazard() such that
there's no particular need for that function to exist. For example,
the following:+ /* + * UPDATE is not currently supported in parallel-mode, so prohibit + * INSERT...ON CONFLICT...DO UPDATE... + * In order to support update, even if only in the leader, some + * further work would need to be done. A mechanism would be needed + * for sharing combo-cids between leader and workers during + * parallel-mode, since for example, the leader might generate a + * combo-cid and it needs to be propagated to the workers. + */ + if (parse->onConflict != NULL && parse->onConflict->action == ONCONFLICT_UPDATE) + return PROPARALLEL_UNSAFE;could be placed in the following block in max_parallel_hazard():
/*
* When we're first invoked on a completely unplanned tree, we must
* recurse into subqueries so to as to locate parallel-unsafe constructs
* anywhere in the tree.
*/
else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}Furthermore, the following:
+ rte = rt_fetch(parse->resultRelation, parse->rtable); + + /* + * The target table is already locked by the caller (this is done in the + * parse/analyze phase). + */ + rel = table_open(rte->relid, NoLock); + (void) rel_max_parallel_hazard_for_modify(rel, parse->commandType, &context); + table_close(rel, NoLock);can itself be wrapped in a function that's called from
max_parallel_hazard() by adding a new block for RangeTblEntry nodes
and passing QTW_EXAMINE_RTES_BEFORE to query_tree_walker().
Thanks, I think those suggestions look good to me.
That brings me to to this part of the hunk:
+ /* + * If there is no underlying SELECT, a parallel table-modification + * operation is not possible (nor desirable). + */ + hasSubQuery = false; + foreach(lc, parse->rtable) + { + rte = lfirst_node(RangeTblEntry, lc); + if (rte->rtekind == RTE_SUBQUERY) + { + hasSubQuery = true; + break; + } + } + if (!hasSubQuery) + return PROPARALLEL_UNSAFE;The justification for this given in:
/messages/by-id/CAJcOf-dF9ohqub_D805k57Y_AuDLeAQfvtaax9SpwjTSEVdiXg@mail.gmail.com
seems to be that the failure of a test case in
partition-concurrent-attach isolation suite is prevented if finding no
subquery RTEs in the query is flagged as parallel unsafe, which in
turn stops max_parallel_hazard_modify() from locking partitions for
safety checks in such cases. But it feels unprincipled to have this
code to work around a specific test case that's failing. I'd rather
edit the failing test case to disable parallel execution as
Tsunakawa-san suggested.
The code was not changed because of the test case (though it was
fortunate that the test case worked after the change).
The code check that you have identified above ensures that the INSERT
has an underlying SELECT, because the planner won't (and shouldn't
anyway) generate a parallel plan for INSERT...VALUES, so there is no
point doing any parallel-safety checks in this case.
It just so happens that the problem test case uses INSERT...VALUES -
and it shouldn't have triggered the parallel-safety checks for
parallel INSERT for this case anyway, because INSERT...VALUES can't
(and shouldn't) be parallelized.
So I will need to keep that check in the code somewhere, to avoid
overhead of parallel-safety checks in the case of INSERT...VALUES.
* Regarding function names:
+static bool trigger_max_parallel_hazard_for_modify(TriggerDesc *trigdesc, + max_parallel_hazard_context *context); +static bool index_expr_max_parallel_hazard_for_modify(Relation rel, + max_parallel_hazard_context *context); +static bool domain_max_parallel_hazard_for_modify(Oid typid, max_parallel_hazard_context *context); +static bool rel_max_parallel_hazard_for_modify(Relation rel, + CmdType command_type, + max_parallel_hazard_context *context)IMO, it would be better to name these
target_rel_trigger_max_parallel_hazard(),
target_rel_index_max_parallel_hazard(), etc. rather than have
_for_modify at the end of these names to better connote that they
check the parallel safety of applying the modify operation to a given
target relation. Also, put these prototypes just below that of
max_parallel_hazard() to have related things close by.Attached please see v15_delta.diff showing the changes suggested above.
OK, sounds reasonable. Thanks for the patch!
* I suspect that the following is broken in light of concurrent
attachment of partitions.+ + /* Recursively check each partition ... */ + pdesc = RelationGetPartitionDesc(rel);I think we'd need to use CreatePartitionDirectory() and retrieve the
PartitionDesc using PartitionDirectoryLookup(). Something we already
do when opening partitions for SELECT planning.* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
prepare q as insert into rp select * from foo where a%2 = 0;
explain execute q;
QUERY PLAN
-------------------------------------------------------------------------------
Gather (cost=1000.00..13041.54 rows=5642 width=0)
Workers Planned: 2
-> Insert on rp (cost=0.00..11477.34 rows=0 width=0)
-> Parallel Seq Scan on foo (cost=0.00..11477.34 rows=2351 width=4)
Filter: ((a % 2) = 0)
(5 rows)-- create a parallel unsafe trigger (that's actually marked so)
directly on a partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();CREATE TRIGGER-- plan still parallel
explain execute q;
QUERY PLAN
-------------------------------------------------------------------------------
Gather (cost=1000.00..13041.54 rows=5642 width=0)
Workers Planned: 2
-> Insert on rp (cost=0.00..11477.34 rows=0 width=0)
-> Parallel Seq Scan on foo (cost=0.00..11477.34 rows=2351 width=4)
Filter: ((a % 2) = 0)
(5 rows)-- and because it is
execute q;
ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement-- OTOH, altering parent correctly discards the parallel plan
create trigger ai_rp after insert on rp for each row execute function
make_table();
explain execute q;
QUERY PLAN
----------------------------------------------------------------
Insert on rp (cost=0.00..19425.00 rows=0 width=0)
-> Seq Scan on foo (cost=0.00..19425.00 rows=5000 width=4)
Filter: ((a % 2) = 0)
(3 rows)It's fair to argue that it would rarely make sense to use PREPARE for
bulk loads, but we need to tighten things up a bit here regardless.
Thanks, looks like you've identified some definite issues in the
partition support and some missing test cases to help detect them.
I'll look into it.
Thanks very much for your review of this and suggestions.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Feb 8, 2021 at 8:13 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Did it actually use a parallel plan in your testing?
When I ran these tests with the Parallel INSERT patch applied, it did
not naturally choose a parallel plan for any of these cases.Yes, these cases pick parallel plan naturally on my test environment.
postgres=# explain verbose insert into testscan select a from x where
a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
----------------------------------------------------------------------
-----------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0
width=0)
-> Parallel Bitmap Heap Scan on public.x1
(cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND (x1.a >
199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808
width=0)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)PSA is my postgresql.conf file, maybe you can have a look. Besides, I didn't
do any parameters tuning in my test session.I reproduced this on my machine.
I think we'd better do "analyze" before insert which helps reproduce this easier.
Like:-----
analyze;
explain analyze verbose insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
-----
OK then.
Can you check if just the underlying SELECTs are run (without INSERT),
is there any performance degradation when compared to a non-parallel
scan?
Regards,
Greg Nancarrow
Fujitsu Australia
postgres=# explain verbose insert into testscan select a from x
where
a<80000 or (a%2=0 and a>199900000);
QUERY PLAN--------------------------------------------------------------------
--
-----------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0
width=0)
-> Parallel Bitmap Heap Scan on public.x1
(cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND
(x1.a >
199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808
width=0)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)PSA is my postgresql.conf file, maybe you can have a look. Besides,
I didn't do any parameters tuning in my test session.I reproduced this on my machine.
I think we'd better do "analyze" before insert which helps reproduce this
easier.
Like:
-----
analyze;
explain analyze verbose insert into testscan select a from x where
a<80000 or (a%2=0 and a>199900000);
-----OK then.
Can you check if just the underlying SELECTs are run (without INSERT), is
there any performance degradation when compared to a non-parallel scan?
It seems there is no performance degradation without insert.
Till now, what I found is that:
With tang's conf, when doing parallel insert, the walrecord is more than serial insert
(IMO, this is the main reason why it has performance degradation)
See the attatchment for the plan info.
I have tried alter the target table to unlogged and
then the performance degradation will not happen any more.
And the additional walrecord seems related to the index on the target table.
If the target table does not have any index, the wal record is the same between parallel plan and serial plan.
Also, it does not have performance degradation without index.
I am still looking at this problem, if someone think of something about it,
It's very grateful to share the knowledge with me.
Best regards,
houzj
On Tue, Feb 9, 2021 at 5:49 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
postgres=# explain verbose insert into testscan select a from x
where
a<80000 or (a%2=0 and a>199900000);
QUERY PLAN--------------------------------------------------------------------
--
-----------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0
width=0)
-> Parallel Bitmap Heap Scan on public.x1
(cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND
(x1.a >
199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808
width=0)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)PSA is my postgresql.conf file, maybe you can have a look. Besides,
I didn't do any parameters tuning in my test session.I reproduced this on my machine.
I think we'd better do "analyze" before insert which helps reproduce this
easier.
Like:
-----
analyze;
explain analyze verbose insert into testscan select a from x where
a<80000 or (a%2=0 and a>199900000);
-----OK then.
Can you check if just the underlying SELECTs are run (without INSERT), is
there any performance degradation when compared to a non-parallel scan?It seems there is no performance degradation without insert.
Till now, what I found is that:
With tang's conf, when doing parallel insert, the walrecord is more than serial insert
(IMO, this is the main reason why it has performance degradation)
See the attatchment for the plan info.I have tried alter the target table to unlogged and
then the performance degradation will not happen any more.And the additional walrecord seems related to the index on the target table.
I think you might want to see which exact WAL records are extra by
using pg_waldump?
--
With Regards,
Amit Kapila.
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
Till now, what I found is that:
With tang's conf, when doing parallel insert, the walrecord is more than serial
insert (IMO, this is the main reason why it has performance degradation) See
the attatchment for the plan info.I have tried alter the target table to unlogged and then the performance
degradation will not happen any more.And the additional walrecord seems related to the index on the target table.
If the target table does not have any index, the wal record is the same between
parallel plan and serial plan.
Also, it does not have performance degradation without index.
[serial]
Insert on public.testscan (cost=3272.20..3652841.26 rows=0 width=0) (actual time=360.474..360.476 rows=0 loops=1)
Buffers: shared hit=392569 read=3 dirtied=934 written=933
WAL: records=260354 bytes=16259841
[parallel]
-> Insert on public.testscan (cost=3272.20..1260119.35 rows=0 width=0) (actual time=378.227..378.229 rows=0 loops=5)
Buffers: shared hit=407094 read=4 dirtied=1085 written=1158
WAL: records=260498 bytes=17019359
More pages are dirtied and written in the parallel execution. Aren't the index and possibly the target table bigger with parallel execution than with serial execution? That may be due to the difference of inserts of index keys.
Regards
Takayuki Tsunakawa
Till now, what I found is that:
With tang's conf, when doing parallel insert, the walrecord is more
than serial insert (IMO, this is the main reason why it has
performance degradation) See the attatchment for the plan info.I have tried alter the target table to unlogged and then the
performance degradation will not happen any more.And the additional walrecord seems related to the index on the target
table.
If the target table does not have any index, the wal record is the
same between parallel plan and serial plan.
Also, it does not have performance degradation without index.[serial]
Insert on public.testscan (cost=3272.20..3652841.26 rows=0 width=0)
(actual time=360.474..360.476 rows=0 loops=1)
Buffers: shared hit=392569 read=3 dirtied=934 written=933
WAL: records=260354 bytes=16259841[parallel]
-> Insert on public.testscan (cost=3272.20..1260119.35 rows=0
width=0) (actual time=378.227..378.229 rows=0 loops=5)
Buffers: shared hit=407094 read=4 dirtied=1085 written=1158
WAL: records=260498 bytes=17019359More pages are dirtied and written in the parallel execution. Aren't the
index and possibly the target table bigger with parallel execution than
with serial execution? That may be due to the difference of inserts of
index keys.
Yes, the table size and index size is bigger with parallel execution.
table and index's size after parallel insert
----------------------
postgres=# select pg_size_pretty(pg_indexes_size('testscan_index'));
pg_size_pretty
----------------
4048 kB
(1 row)
postgres=#
postgres=# select pg_size_pretty(pg_relation_size('testscan_index'));
pg_size_pretty
----------------
4768 kB
(1 row)
----------------------
table and index's size after serial insert
----------------------
postgres=# select pg_size_pretty(pg_indexes_size('testscan_index'));
pg_size_pretty
----------------
2864 kB
(1 row)
postgres=# select pg_size_pretty(pg_relation_size('testscan_index'));
pg_size_pretty
----------------
4608 kB
----------------------
To Amit:
I think you might want to see which exact WAL records are extra by using pg_waldump?
Yes, thanks for the hint, I was doing that and the result is as follow:
Heap wal record is the same between parallel and serial: (129999 which is the number count of the query result).
parallel Btree walrecord(130500 record):
----------------------
INSERT_LEAF:129500
INSERT_UPPER:497
SPLIT_L:172
SPLIT_R:328
INSERT_POST:0
DEDUP:0
VACUUM:0
DELETE:0
MARK_PAGE_HALFDEAD:0
UNLINK_PAGE:0
UNLINK_PAGE_META:0
NEWROOT:3
REUSE_PAGE:0
META_CLEANUP:0
----------------------
serial Btree walrecord(130355 record):
----------------------
INSERT_LEAF:129644
INSERT_UPPER:354
SPLIT_L:0
SPLIT_R:355
INSERT_POST:0
DEDUP:0
VACUUM:0
DELETE:0
MARK_PAGE_HALFDEAD:0
UNLINK_PAGE:0
UNLINK_PAGE_META:0
NEWROOT:2
REUSE_PAGE:0
META_CLEANUP:0
----------------------
IMO, due to the difference of inserts with parallel execution,
the btree insert's cost is more than serial.
At the same time, the parallel does not have a huge performance gain with bitmapscan,
So the extra cost of btree index will result in performance degradation.
Does it make sense ?
Best regards,
Houzj
On Tue, Feb 9, 2021 at 10:30 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
That brings me to to this part of the hunk:
+ /* + * If there is no underlying SELECT, a parallel table-modification + * operation is not possible (nor desirable). + */ + hasSubQuery = false; + foreach(lc, parse->rtable) + { + rte = lfirst_node(RangeTblEntry, lc); + if (rte->rtekind == RTE_SUBQUERY) + { + hasSubQuery = true; + break; + } + } + if (!hasSubQuery) + return PROPARALLEL_UNSAFE;The justification for this given in:
/messages/by-id/CAJcOf-dF9ohqub_D805k57Y_AuDLeAQfvtaax9SpwjTSEVdiXg@mail.gmail.com
seems to be that the failure of a test case in
partition-concurrent-attach isolation suite is prevented if finding no
subquery RTEs in the query is flagged as parallel unsafe, which in
turn stops max_parallel_hazard_modify() from locking partitions for
safety checks in such cases. But it feels unprincipled to have this
code to work around a specific test case that's failing. I'd rather
edit the failing test case to disable parallel execution as
Tsunakawa-san suggested.The code was not changed because of the test case (though it was
fortunate that the test case worked after the change).
Ah, I misread then, sorry.
The code check that you have identified above ensures that the INSERT
has an underlying SELECT, because the planner won't (and shouldn't
anyway) generate a parallel plan for INSERT...VALUES, so there is no
point doing any parallel-safety checks in this case.
It just so happens that the problem test case uses INSERT...VALUES -
and it shouldn't have triggered the parallel-safety checks for
parallel INSERT for this case anyway, because INSERT...VALUES can't
(and shouldn't) be parallelized.
AFAICS, max_parallel_hazard() path never bails from doing further
safety checks based on anything other than finding a query component
whose hazard level crosses context->max_interesting. You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.
If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Wed, Jan 6, 2021 at 7:39 PM Antonin Houska <ah@cybertec.at> wrote:
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,17 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}/* Now fix the Plan tree */ - return set_plan_refs(root, plan, rtoffset); + finalPlan = set_plan_refs(root, plan, rtoffset); + if (finalPlan != NULL && IsA(finalPlan, Gather)) + { + Plan *subplan = outerPlan(finalPlan); + + if (IsA(subplan, ModifyTable) && castNode(ModifyTable, subplan)->returningLists != NULL) + { + finalPlan->targetlist = copyObject(subplan->targetlist); + } + } + return finalPlan; }I'm not sure if the problem of missing targetlist should be handled here (BTW,
NIL is the constant for an empty list, not NULL). Obviously this is a
consequence of the fact that the ModifyTable node has no regular targetlist.Actually I don't quite understand why (in the current master branch) the
targetlist initialized in set_plan_refs()/*
* Set up the visible plan targetlist as being the same as
* the first RETURNING list. This is for the use of
* EXPLAIN; the executor won't pay any attention to the
* targetlist. We postpone this step until here so that
* we don't have to do set_returning_clause_references()
* twice on identical targetlists.
*/
splan->plan.targetlist = copyObject(linitial(newRL));is not used. Instead, ExecInitModifyTable() picks the first returning list
again:/*
* Initialize result tuple slot and assign its rowtype using the first
* RETURNING list. We assume the rest will look the same.
*/
mtstate->ps.plan->targetlist = (List *) linitial(node->returningLists);So if you set the targetlist in create_modifytable_plan() (according to
best_path->returningLists), or even in create_modifytable_path(), and ensure
that it gets propagated to the Gather node (generate_gather_pahs currently
uses rel->reltarget), then you should no longer need to tweak
setrefs.c. Moreover, ExecInitModifyTable() would no longer need to set the
targetlist. However I don't guarantee that this is the best approach - some
planner expert should speak up.
I've had a bit closer look at this particular issue.
I can see what you mean about the ModifyTable targetlist (that is set
in set_plan_refs()) getting overwritten by ExecInitModifyTable() -
which also contradicts the comment in set_plan_refs() that claims the
targetlist being set is used by EXPLAIN (which it is not). So the
current Postgres master branch does seem to be broken in that respect.
I did try your suggestion (and also remove my tweak), but I found that
in the T_Gather case of set_plan_refs() it does some processing (see
set_upper_references()) of the current Gather targetlist and subplan's
targetlist (and will then overwrite the Gather targetlist after that),
but in doing that processing it produces the error:
ERROR: variable not found in subplan target list
I think that one of the fundamental problems is that, up to now,
ModifyTable has always been the top node in a (non-parallel) plan, but
now for Parallel INSERT we have a Gather node with ModifyTable in its
subplan. So the expected order of processing and node configuration
has changed.
For the moment (until perhaps a planner expert DOES speak up) I've
parked my temporary "fix" at the bottom of set_plan_refs(), simply
pointing the Gather node's targetlist to subplan's ModifyTable
targetlist.
if (nodeTag(plan) == T_Gather)
{
Plan *subplan = plan->lefttree;
if (IsA(subplan, ModifyTable) &&
castNode(ModifyTable, subplan)->returningLists != NIL)
{
plan->targetlist = subplan->targetlist;
}
}
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 10, 2021 at 2:39 PM Amit Langote <amitlangote09@gmail.com> wrote:
The code check that you have identified above ensures that the INSERT
has an underlying SELECT, because the planner won't (and shouldn't
anyway) generate a parallel plan for INSERT...VALUES, so there is no
point doing any parallel-safety checks in this case.
It just so happens that the problem test case uses INSERT...VALUES -
and it shouldn't have triggered the parallel-safety checks for
parallel INSERT for this case anyway, because INSERT...VALUES can't
(and shouldn't) be parallelized.AFAICS, max_parallel_hazard() path never bails from doing further
safety checks based on anything other than finding a query component
whose hazard level crosses context->max_interesting.
It's parallel UNSAFE because it's not safe or even possible to have a
parallel plan for this.
(as UNSAFE is the max hazard level, no point in referencing
context->max_interesting).
And there are existing cases of bailing out and not doing further
safety checks (even your v15_delta.diff patch placed code - for
bailing out on "ON CONFLICT ... DO UPDATE" - underneath one such
existing case in max_parallel_hazard_walker()):
else if (IsA(node, Query))
{
Query *query = (Query *) node;
/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}
You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.
There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".
Regards,
Greg Nancarrow
Fujitsu Australia
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
table and index's size after parallel insert
----------------------
postgres=# select pg_size_pretty(pg_indexes_size('testscan_index'));
pg_size_pretty
----------------
4048 kB
(1 row)postgres=# select pg_size_pretty(pg_relation_size('testscan_index'));
pg_size_pretty
----------------
4768 kB
(1 row)
Which of the above shows the table size? What does pg_indexes_size() against an index (testscan_index) return?
IMO, due to the difference of inserts with parallel execution,
the btree insert's cost is more than serial.At the same time, the parallel does not have a huge performance gain with
bitmapscan,
So the extra cost of btree index will result in performance degradation.
How did you know that the parallelism didn't have a huge performance gain with bitmap scan?
[serial]
-> Bitmap Heap Scan on public.x (cost=3272.20..3652841.26 rows=79918 width=8) (actual time=8.096..41.005 rows=129999 loops=1)
[parallel]
-> Parallel Bitmap Heap Scan on public.x (cost=3272.20..1260119.35 rows=19980 width=8) (actual time=5.832..14.787 rows=26000 loops=5)
Regards
Takayuki Tsunakawa
----------------------
postgres=# select pg_size_pretty(pg_indexes_size('testscan_index'));
pg_size_pretty
----------------
4048 kB
(1 row)postgres=# select pg_size_pretty(pg_relation_size('testscan_index'));
pg_size_pretty
----------------
4768 kB
(1 row)Which of the above shows the table size? What does pg_indexes_size()
against an index (testscan_index) return?
Sorry, Maybe the tablename is a little confused,
but 'testscan_index' is actually a table's name.
pg_indexes_size will return the index's size attatched to the table.
pg_relation_size will return the table's size.
Did I miss something ?
IMO, due to the difference of inserts with parallel execution, the
btree insert's cost is more than serial.At the same time, the parallel does not have a huge performance gain
with bitmapscan, So the extra cost of btree index will result in
performance degradation.How did you know that the parallelism didn't have a huge performance gain
with bitmap scan?[serial]
-> Bitmap Heap Scan on public.x (cost=3272.20..3652841.26 rows=79918
width=8) (actual time=8.096..41.005 rows=129999 loops=1)[parallel]
-> Parallel Bitmap Heap Scan on public.x
(cost=3272.20..1260119.35 rows=19980 width=8) (actual time=5.832..14.787
rows=26000 loops=5)
I tested the case without insert(Just the query use bitmapscan):
[serial]:
postgres=# explain analyze select a from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------
Bitmap Heap Scan on x (cost=3258.59..3647578.53 rows=81338 width=4) (actual time=8.091..34.222 rows=129999 loops=1)
Recheck Cond: ((a < 80000) OR (a > 199900000))
Filter: ((a < 80000) OR (((a % 2) = 0) AND (a > 199900000)))
Rows Removed by Filter: 50000
Heap Blocks: exact=975
-> BitmapOr (cost=3258.59..3258.59 rows=173971 width=0) (actual time=7.964..7.965 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1495.11 rows=80872 width=0) (actual time=3.451..3.451 rows=79999 loops=1)
Index Cond: (a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1722.81 rows=93099 width=0) (actual time=4.513..4.513 rows=100000 loops=1)
Index Cond: (a > 199900000)
Planning Time: 0.108 ms
Execution Time: 38.136 ms
[parallel]
postgres=# explain analyze select a from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------
Gather (cost=4258.59..1266704.42 rows=81338 width=4) (actual time=9.177..22.592 rows=129999 loops=1)
Workers Planned: 4
Workers Launched: 4
-> Parallel Bitmap Heap Scan on x (cost=3258.59..1257570.62 rows=20334 width=4) (actual time=6.402..12.882 rows=26000 loops=5)
Recheck Cond: ((a < 80000) OR (a > 199900000))
Filter: ((a < 80000) OR (((a % 2) = 0) AND (a > 199900000)))
Rows Removed by Filter: 10000
Heap Blocks: exact=1
-> BitmapOr (cost=3258.59..3258.59 rows=173971 width=0) (actual time=8.785..8.786 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1495.11 rows=80872 width=0) (actual time=3.871..3.871 rows=79999 loops=1)
Index Cond: (a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1722.81 rows=93099 width=0) (actual time=4.914..4.914 rows=100000 loops=1)
Index Cond: (a > 199900000)
Planning Time: 0.158 ms
Execution Time: 26.951 ms
(15 rows)
It did have performance gain, but I think it's not huge enough to ignore the extra's index cost.
What do you think ?
Best regards,
houzj
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
Sorry, Maybe the tablename is a little confused,
but 'testscan_index' is actually a table's name.
...
Did I miss something ?
No, I don't think so. I just wanted to know the facts correctly. Your EXPLAIN output shows that the target table is testscan as follows. How does testscan_index relate to testscan?
-> Insert on public.testscan (cost=3272.20..1260119.35 rows=0 width=0) (actual time=378.227..378.229 rows=0 loops=5)
It did have performance gain, but I think it's not huge enough to ignore the
extra's index cost.
What do you think ?
Yes... as you suspect, I'm afraid the benefit from parallel bitmap scan may not compensate for the loss of the parallel insert operation.
The loss is probably due to 1) more index page splits, 2) more buffer writes (table and index), and 3) internal locks for things such as relation extension and page content protection. To investigate 3), we should want something like [1]Add accumulated statistics for wait event https://commitfest.postgresql.org/28/2332/, which tells us the wait event statistics (wait count and time for each wait event) per session or across the instance like Oracke, MySQL and EDB provides. I want to continue this in the near future.
[1]: Add accumulated statistics for wait event https://commitfest.postgresql.org/28/2332/
Add accumulated statistics for wait event
https://commitfest.postgresql.org/28/2332/
Regards
Takayuki Tsunakawa
From: tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com>
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
It did have performance gain, but I think it's not huge enough to ignore the
extra's index cost.
What do you think ?Yes... as you suspect, I'm afraid the benefit from parallel bitmap scan may not
compensate for the loss of the parallel insert operation.The loss is probably due to 1) more index page splits, 2) more buffer writes
(table and index), and 3) internal locks for things such as relation extension
and page content protection. To investigate 3), we should want something
like [1], which tells us the wait event statistics (wait count and time for each
wait event) per session or across the instance like Oracke, MySQL and EDB
provides. I want to continue this in the near future.
What would the result look like if you turn off parallel_leader_participation? If the leader is freed from reading/writing the table and index, the index page splits and internal lock contention may decrease enough to recover part of the loss.
https://www.postgresql.org/docs/devel/parallel-plans.html
"In a parallel bitmap heap scan, one process is chosen as the leader. That process performs a scan of one or more indexes and builds a bitmap indicating which table blocks need to be visited. These blocks are then divided among the cooperating processes as in a parallel sequential scan. In other words, the heap scan is performed in parallel, but the underlying index scan is not."
BTW, the following sentences seem to be revisited, because "the work to be done" is not the same for parallel INSERT as for serial INSERT - the order of rows stored, table and index sizes, and what else?
https://www.postgresql.org/docs/devel/using-explain.html#USING-EXPLAIN-ANALYZE
"It's worth noting that although the data-modifying node can take a considerable amount of run time (here, it's consuming the lion's share of the time), the planner does not currently add anything to the cost estimates to account for that work. That's because the work to be done is the same for every correct query plan, so it doesn't affect planning decisions."
Regards
Takayuki Tsunakawa
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 10, 2021 at 2:39 PM Amit Langote <amitlangote09@gmail.com> wrote:
The code check that you have identified above ensures that the INSERT
has an underlying SELECT, because the planner won't (and shouldn't
anyway) generate a parallel plan for INSERT...VALUES, so there is no
point doing any parallel-safety checks in this case.
It just so happens that the problem test case uses INSERT...VALUES -
and it shouldn't have triggered the parallel-safety checks for
parallel INSERT for this case anyway, because INSERT...VALUES can't
(and shouldn't) be parallelized.AFAICS, max_parallel_hazard() path never bails from doing further
safety checks based on anything other than finding a query component
whose hazard level crosses context->max_interesting.It's parallel UNSAFE because it's not safe or even possible to have a
parallel plan for this.
(as UNSAFE is the max hazard level, no point in referencing
context->max_interesting).
And there are existing cases of bailing out and not doing further
safety checks (even your v15_delta.diff patch placed code - for
bailing out on "ON CONFLICT ... DO UPDATE" - underneath one such
existing case in max_parallel_hazard_walker()):else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}
In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().
You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".
I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.
--
Amit Langote
EDB: http://www.enterprisedb.com
From: Amit Langote <amitlangote09@gmail.com>
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com>
wrote:There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.
In terms of competitiveness, Oracle does not run INSERT VALUES in parallel:
"The INSERT VALUES statement is never executed in parallel."
And SQL Server doesn't either:
"Starting with SQL Server 2016 (13.x) and database compatibility level 130, the INSERT … SELECT statement can be executed in parallel when inserting into heaps or clustered columnstore indexes (CCI), and using the TABLOCK hint."
Regards
Takayuki Tsunakawa
On Wed, Feb 10, 2021 at 5:03 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
From: Amit Langote <amitlangote09@gmail.com>
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com>
wrote:There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.In terms of competitiveness, Oracle does not run INSERT VALUES in parallel:
"The INSERT VALUES statement is never executed in parallel."
And SQL Server doesn't either:
"Starting with SQL Server 2016 (13.x) and database compatibility level 130, the INSERT … SELECT statement can be executed in parallel when inserting into heaps or clustered columnstore indexes (CCI), and using the TABLOCK hint."
Just to be clear, I'm not suggesting that we should put effort into
making INSERT ... VALUES run in parallel. I'm just raising my concern
about embedding the assumption in max_parallel_hazard() that it will
never make sense to do so.
Although, maybe there are other more pressing concerns to resolve, so
I will not insist too much on doing anything about this.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Wed, Feb 10, 2021 at 1:00 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 10, 2021 at 2:39 PM Amit Langote <amitlangote09@gmail.com> wrote:
The code check that you have identified above ensures that the INSERT
has an underlying SELECT, because the planner won't (and shouldn't
anyway) generate a parallel plan for INSERT...VALUES, so there is no
point doing any parallel-safety checks in this case.
It just so happens that the problem test case uses INSERT...VALUES -
and it shouldn't have triggered the parallel-safety checks for
parallel INSERT for this case anyway, because INSERT...VALUES can't
(and shouldn't) be parallelized.AFAICS, max_parallel_hazard() path never bails from doing further
safety checks based on anything other than finding a query component
whose hazard level crosses context->max_interesting.It's parallel UNSAFE because it's not safe or even possible to have a
parallel plan for this.
(as UNSAFE is the max hazard level, no point in referencing
context->max_interesting).
And there are existing cases of bailing out and not doing further
safety checks (even your v15_delta.diff patch placed code - for
bailing out on "ON CONFLICT ... DO UPDATE" - underneath one such
existing case in max_parallel_hazard_walker()):else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.
I think it is more of a case where neither is a need for parallelism
nor we want to support parallelism of it. The other possibility for
such a check could be at some earlier phase say in standard_planner
[1]: standard_planner() { .. if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster && parse->commandType == CMD_SELECT && !parse->hasModifyingCTE && max_parallel_workers_per_gather > 0 && !IsParallelWorker()) { /* all the cheap tests pass, so scan the query tree */ glob->maxParallelHazard = max_parallel_hazard(parse); glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); } else { /* skip the query tree scan, just assume it's unsafe */ glob->maxParallelHazard = PROPARALLEL_UNSAFE; glob->parallelModeOK = false; }
parallelism (I think the check for 'parse->hasModifyingCTE' is quite
similar). If you see in that check as well we just assume other
operations to be in the category of parallel-unsafe. I think we should
rule out such cases earlier than later. Do you have better ideas than
what Greg has done to avoid parallelism for such cases?
[1]: standard_planner() { .. if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster && parse->commandType == CMD_SELECT && !parse->hasModifyingCTE && max_parallel_workers_per_gather > 0 && !IsParallelWorker()) { /* all the cheap tests pass, so scan the query tree */ glob->maxParallelHazard = max_parallel_hazard(parse); glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE); } else { /* skip the query tree scan, just assume it's unsafe */ glob->maxParallelHazard = PROPARALLEL_UNSAFE; glob->parallelModeOK = false; }
standard_planner()
{
..
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
parse->commandType == CMD_SELECT &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}
--
With Regards,
Amit Kapila.
On Wed, Feb 10, 2021 at 11:13 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
The loss is probably due to 1) more index page splits, 2) more buffer writes (table and index), and 3) internal locks for things such as relation extension and page content protection. To investigate 3), we should want something like [1], which tells us the wait event statistics (wait count and time for each wait event) per session or across the instance like Oracke, MySQL and EDB provides. I want to continue this in the near future.
I think we might mitigate such losses with a different patch where we
can do a bulk insert for Insert .. Select something like we are
discussing in the other thread [1]/messages/by-id/20200508072545.GA9701@telsasoft.com. I wonder if these performance
characteristics are due to the reason of underlying bitmap heap scan.
What are the results if disable the bitmap heap scan(Set
enable_bitmapscan = off)? If that happens to be true, then we might
also want to consider if in some way we can teach parallel insert to
cost more in such cases. Another thing we can try is to integrate a
parallel-insert patch with the patch on another thread [1]/messages/by-id/20200508072545.GA9701@telsasoft.com and see if
that makes any difference but not sure if we want to go there at this
stage unless it is simple to try that out?
[1]: /messages/by-id/20200508072545.GA9701@telsasoft.com
--
With Regards,
Amit Kapila.
On Wed, Feb 10, 2021 at 5:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 10, 2021 at 1:00 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
It's parallel UNSAFE because it's not safe or even possible to have a
parallel plan for this.
(as UNSAFE is the max hazard level, no point in referencing
context->max_interesting).
And there are existing cases of bailing out and not doing further
safety checks (even your v15_delta.diff patch placed code - for
bailing out on "ON CONFLICT ... DO UPDATE" - underneath one such
existing case in max_parallel_hazard_walker()):else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.I think it is more of a case where neither is a need for parallelism
nor we want to support parallelism of it. The other possibility for
such a check could be at some earlier phase say in standard_planner
[1] where we are doing checks for other constructs where we don't want
parallelism (I think the check for 'parse->hasModifyingCTE' is quite
similar). If you see in that check as well we just assume other
operations to be in the category of parallel-unsafe. I think we should
rule out such cases earlier than later. Do you have better ideas than
what Greg has done to avoid parallelism for such cases?[1] -
standard_planner()
{
..
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
parse->commandType == CMD_SELECT &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}
Yeah, maybe having the block I was commenting on, viz.:
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
before the standard_planner() block you quoted might be a good idea.
So something like this:
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ rangeTablehasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ rangeTableHasSubQuery = true;
+ break;
+ }
+ }
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
(IsModifySupportedInParallelMode(parse->commandType) &&
rangeTableHasSubQuery)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}
--
Amit Langote
EDB: http://www.enterprisedb.com
From: Amit Langote <amitlangote09@gmail.com>
Just to be clear, I'm not suggesting that we should put effort into
making INSERT ... VALUES run in parallel. I'm just raising my concern
about embedding the assumption in max_parallel_hazard() that it will
never make sense to do so.
I'm sorry I misunderstood your suggestion. So, you're suggesting that it may be better to place the VALUES existence check outside max_parallel_hazard(). (I may be a bit worried if I may misunderstanding in a different way.)
The description of max_parallel_hazard() gave me an impression that this is the right place to check VALUES, because its role can be paraphrased in simpler words like "Tell you if the given Query is safe for parallel execution." In that regard, the standard_planner()'s if conditions that check Query->commandType and Query->hasModifyingCTE can be moved into max_parallel_hazard() too.
But looking closer to the description, it says "Returns the worst function hazard." Function hazard? Should this function only check functions? Or do we want to modify this description and get max_parallel_hazard() to provide more service?
/*
* max_parallel_hazard
* Find the worst parallel-hazard level in the given query
*
* Returns the worst function hazard property (the earliest in this list:
* PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE) that can
* be found in the given parsetree. We use this to find out whether the query
* can be parallelized at all. The caller will also save the result in
* PlannerGlobal so as to short-circuit checks of portions of the querytree
* later, in the common case where everything is SAFE.
*/
char
max_parallel_hazard(Query *parse)
Regards
Takayuki Tsunakawa
else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().You're trying to
add something that bails based on second-guessing that a parallel
path would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe
we should try to somehow make the check less expensive. I
remember reading somewhere in the thread about caching the result
of this check in relcache, but haven't closely studied the feasibilityof doing so.
There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES"
case (apart from the fact that spawning off parallel workers to
insert that data would almost always result in worse performance
than a non-parallel plan...) The division of work for parallel
workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.I think it is more of a case where neither is a need for parallelism nor
we want to support parallelism of it. The other possibility for such a check
could be at some earlier phase say in standard_planner [1] where we are
doing checks for other constructs where we don't want parallelism (I think
the check for 'parse->hasModifyingCTE' is quite similar). If you see in
that check as well we just assume other operations to be in the category
of parallel-unsafe. I think we should rule out such cases earlier than later.
Do you have better ideas than what Greg has done to avoid parallelism for
such cases?[1] -
standard_planner()
{
..
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster &&
parse->commandType == CMD_SELECT &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse); parallelModeOK =
glob->(glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE; parallelModeOK = false;
}
+1.
In the current parallel_dml option patch. I put this check and some high-level check in a separate function called is_parallel_possible_for_modify.
- * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) enable_parallel_dml is off
+ * 2) UPDATE or DELETE command
+ * 3) INSERT...ON CONFLICT...DO UPDATE
+ * 4) INSERT without SELECT on a relation
+ * 5) the reloption parallel_dml_enabled is not set for the target table
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
*/
+bool
+is_parallel_possible_for_modify(Query *parse)
And I put the function at earlier place like the following:
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
- (enable_parallel_dml &&
- IsModifySupportedInParallelMode(parse->commandType))) &&
+ is_parallel_possible_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
If this looks good, maybe we can merge this change.
Best regards,
houzj
On Wed, Feb 10, 2021 at 5:52 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
From: Amit Langote <amitlangote09@gmail.com>
Just to be clear, I'm not suggesting that we should put effort into
making INSERT ... VALUES run in parallel. I'm just raising my concern
about embedding the assumption in max_parallel_hazard() that it will
never make sense to do so.I'm sorry I misunderstood your suggestion. So, you're suggesting that it may be better to place the VALUES existence check outside max_parallel_hazard(). (I may be a bit worried if I may misunderstanding in a different way.)
To add context to my comments, here's the block of code in the patch I
was commenting on:
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return PROPARALLEL_UNSAFE;
For a modification query, this makes max_parallel_hazard() return that
it is unsafe to parallelize the query because it doesn't contain a
subquery RTE, or only contains a VALUES RTE.
I was trying to say that inside max_parallel_hazard() seems to be a
wrong place to reject parallelism for modification if only because
there are no subquery RTEs in the query. Although now I'm thinking
that maybe it's okay as long as it's appropriately placed. I shared
one suggestion in my reply to Amit K's email.
The description of max_parallel_hazard() gave me an impression that this is the right place to check VALUES, because its role can be paraphrased in simpler words like "Tell you if the given Query is safe for parallel execution."
In that regard, the standard_planner()'s if conditions that check Query->commandType and Query->hasModifyingCTE can be moved into max_parallel_hazard() too.
But looking closer to the description, it says "Returns the worst function hazard." Function hazard? Should this function only check functions? Or do we want to modify this description and get max_parallel_hazard() to provide more service?
Yeah, updating the description to be more general may make sense.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Wed, Feb 10, 2021 at 5:50 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 10, 2021 at 5:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 10, 2021 at 1:00 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 10, 2021 at 1:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
It's parallel UNSAFE because it's not safe or even possible to have a
parallel plan for this.
(as UNSAFE is the max hazard level, no point in referencing
context->max_interesting).
And there are existing cases of bailing out and not doing further
safety checks (even your v15_delta.diff patch placed code - for
bailing out on "ON CONFLICT ... DO UPDATE" - underneath one such
existing case in max_parallel_hazard_walker()):else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().You're trying to
add something that bails based on second-guessing that a parallel path
would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe we
should try to somehow make the check less expensive. I remember
reading somewhere in the thread about caching the result of this check
in relcache, but haven't closely studied the feasibility of doing so.There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES" case
(apart from the fact that spawning off parallel workers to insert that
data would almost always result in worse performance than a
non-parallel plan...)
The division of work for parallel workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.I think it is more of a case where neither is a need for parallelism
nor we want to support parallelism of it. The other possibility for
such a check could be at some earlier phase say in standard_planner
[1] where we are doing checks for other constructs where we don't want
parallelism (I think the check for 'parse->hasModifyingCTE' is quite
similar). If you see in that check as well we just assume other
operations to be in the category of parallel-unsafe. I think we should
rule out such cases earlier than later. Do you have better ideas than
what Greg has done to avoid parallelism for such cases?[1] -
standard_planner()
{
..
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
parse->commandType == CMD_SELECT &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}Yeah, maybe having the block I was commenting on, viz.:
+ /* + * If there is no underlying SELECT, a parallel table-modification + * operation is not possible (nor desirable). + */ + hasSubQuery = false; + foreach(lc, parse->rtable) + { + rte = lfirst_node(RangeTblEntry, lc); + if (rte->rtekind == RTE_SUBQUERY) + { + hasSubQuery = true; + break; + } + } + if (!hasSubQuery) + return PROPARALLEL_UNSAFE;before the standard_planner() block you quoted might be a good idea.
So something like this:+ /* + * If there is no underlying SELECT, a parallel table-modification + * operation is not possible (nor desirable). + */ + rangeTablehasSubQuery = false; + foreach(lc, parse->rtable) + { + rte = lfirst_node(RangeTblEntry, lc); + if (rte->rtekind == RTE_SUBQUERY) + { + rangeTableHasSubQuery = true; + break; + } + }if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
(parse->commandType == CMD_SELECT ||
(IsModifySupportedInParallelMode(parse->commandType) &&
rangeTableHasSubQuery)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE;
glob->parallelModeOK = false;
}
On second thought, maybe we could even put the hasSubQuery-based
short-circuit in the following block of max_parallel_hazard_walker():
/*
* When we're first invoked on a completely unplanned tree, we must
* recurse into subqueries so to as to locate parallel-unsafe constructs
* anywhere in the tree.
*/
else if (IsA(node, Query))
{
Query *query = (Query *) node;
/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}
Also, update the comment to mention we bail out if (!hasSubQuery) as a
special case.
--
Amit Langote
EDB: http://www.enterprisedb.com
What are the results if disable the bitmap heap scan(Set enable_bitmapscan
= off)? If that happens to be true, then we might also want to consider
if in some way we can teach parallel insert to cost more in such cases.
Another thing we can try is to integrate a parallel-insert patch with the
patch on another thread [1] and see if that makes any difference but not
sure if we want to go there at this stage unless it is simple to try that
out?
If we diable bitmapscan, the performance degradation seems will not happen.
[Parallel]
postgres=# explain (ANALYZE, BUFFERS, VERBOSE, WAL) insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------
Gather (cost=1000.00..2090216.68 rows=81338 width=0) (actual time=0.226..5488.455 rows=0 loops=1)
Workers Planned: 4
Workers Launched: 4
Buffers: shared hit=393364 read=1079535 dirtied=984 written=1027
WAL: records=260400 bytes=16549513
-> Insert on public.testscan (cost=0.00..2081082.88 rows=0 width=0) (actual time=5483.113..5483.114 rows=0 loops=4)
Buffers: shared hit=393364 read=1079535 dirtied=984 written=1027
WAL: records=260400 bytes=16549513
Worker 0: actual time=5483.116..5483.117 rows=0 loops=1
Buffers: shared hit=36306 read=264288 dirtied=100 written=49
WAL: records=23895 bytes=1575860
Worker 1: actual time=5483.220..5483.222 rows=0 loops=1
Buffers: shared hit=39750 read=280476 dirtied=101 written=106
WAL: records=26141 bytes=1685083
Worker 2: actual time=5482.844..5482.845 rows=0 loops=1
Buffers: shared hit=38660 read=263713 dirtied=105 written=250
WAL: records=25318 bytes=1657396
Worker 3: actual time=5483.272..5483.274 rows=0 loops=1
Buffers: shared hit=278648 read=271058 dirtied=678 written=622
WAL: records=185046 bytes=11631174
-> Parallel Seq Scan on public.x (cost=0.00..2081082.88 rows=20334 width=8) (actual time=4001.641..5287.248 rows=32500 loops=4)
Output: x.a, NULL::integer
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199900000)))
Rows Removed by Filter: 49967500
Buffers: shared hit=1551 read=1079531
Worker 0: actual time=5335.456..5340.757 rows=11924 loops=1
Buffers: shared hit=281 read=264288
Worker 1: actual time=5335.559..5341.766 rows=13049 loops=1
Buffers: shared hit=281 read=280476
Worker 2: actual time=5335.534..5340.964 rows=12636 loops=1
Buffers: shared hit=278 read=263712
Worker 3: actual time=0.015..5125.503 rows=92390 loops=1
Buffers: shared hit=711 read=271055
Planning:
Buffers: shared hit=19
Planning Time: 0.175 ms
Execution Time: 5488.493 ms
[Serial]
postgres=# explain (ANALYZE, BUFFERS, VERBOSE, WAL) insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------
Insert on public.testscan (cost=0.00..5081085.52 rows=0 width=0) (actual time=19311.642..19311.643 rows=0 loops=1)
Buffers: shared hit=392485 read=1079694 dirtied=934 written=933
WAL: records=260354 bytes=16259841
-> Seq Scan on public.x (cost=0.00..5081085.52 rows=81338 width=8) (actual time=0.010..18997.317 rows=129999 loops=1)
Output: x.a, NULL::integer
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199900000)))
Rows Removed by Filter: 199870001
Buffers: shared hit=1391 read=1079691
Planning:
Buffers: shared hit=10
Planning Time: 0.125 ms
Execution Time: 19311.700 ms
Best regards,
houzj
It did have performance gain, but I think it's not huge enough to
ignore the extra's index cost.
What do you think ?Yes... as you suspect, I'm afraid the benefit from parallel bitmap
scan may not compensate for the loss of the parallel insert operation.The loss is probably due to 1) more index page splits, 2) more buffer
writes (table and index), and 3) internal locks for things such as
relation extension and page content protection. To investigate 3), we
should want something like [1], which tells us the wait event
statistics (wait count and time for each wait event) per session or
across the instance like Oracke, MySQL and EDB provides. I want tocontinue this in the near future.
What would the result look like if you turn off
parallel_leader_participation? If the leader is freed from
reading/writing the table and index, the index page splits and internal
lock contention may decrease enough to recover part of the loss.https://www.postgresql.org/docs/devel/parallel-plans.html
"In a parallel bitmap heap scan, one process is chosen as the leader. That
process performs a scan of one or more indexes and builds a bitmap indicating
which table blocks need to be visited. These blocks are then divided among
the cooperating processes as in a parallel sequential scan. In other words,
the heap scan is performed in parallel, but the underlying index scan is
not."
If I disable parallel_leader_participation.
For max_parallel_workers_per_gather = 4, It still have performance degradation.
For max_parallel_workers_per_gather = 2, the performance degradation will not happen in most of the case.
There is sometimes a noise(performance degradation), but most of result(about 80%) is good.
Best regards,
houzj
On Wed, Feb 10, 2021 at 8:59 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
else if (IsA(node, Query))
{
Query *query = (Query *) node;/* SELECT FOR UPDATE/SHARE must be treated as unsafe */
if (query->rowMarks != NULL)
{
context->max_hazard = PROPARALLEL_UNSAFE;
return true;
}In my understanding, the max_parallel_hazard() query tree walk is to
find constructs that are parallel unsafe in that they call code that
can't run in parallel mode. For example, FOR UPDATE/SHARE on
traditional heap AM tuples calls AssignTransactionId() which doesn't
support being called in parallel mode. Likewise ON CONFLICT ... DO
UPDATE calls heap_update() which doesn't support parallelism. I'm not
aware of any such hazards downstream of ExecValuesScan().You're trying to
add something that bails based on second-guessing that a parallel
path would not be chosen, which I find somewhat objectionable.If the main goal of bailing out is to avoid doing the potentially
expensive modification safety check on the target relation, maybe
we should try to somehow make the check less expensive. I
remember reading somewhere in the thread about caching the result
of this check in relcache, but haven't closely studied the feasibilityof doing so.
There's no "second-guessing" involved here.
There is no underlying way of dividing up the VALUES data of
"INSERT...VALUES" amongst the parallel workers, even if the planner
was updated to produce a parallel-plan for the "INSERT...VALUES"
case (apart from the fact that spawning off parallel workers to
insert that data would almost always result in worse performance
than a non-parallel plan...) The division of work for parallel
workers is part of the table AM
(scan) implementation, which is not invoked for "INSERT...VALUES".I don't disagree that the planner would not normally assign a parallel
path simply to pull values out of a VALUES list mentioned in the
INSERT command, but deciding something based on the certainty of it in
an earlier planning phase seems odd to me. Maybe that's just me
though.I think it is more of a case where neither is a need for parallelism nor
we want to support parallelism of it. The other possibility for such a check
could be at some earlier phase say in standard_planner [1] where we are
doing checks for other constructs where we don't want parallelism (I think
the check for 'parse->hasModifyingCTE' is quite similar). If you see in
that check as well we just assume other operations to be in the category
of parallel-unsafe. I think we should rule out such cases earlier than later.
Do you have better ideas than what Greg has done to avoid parallelism for
such cases?[1] -
standard_planner()
{
..
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster &&
parse->commandType == CMD_SELECT &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
glob->maxParallelHazard = max_parallel_hazard(parse); parallelModeOK =
glob->(glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
{
/* skip the query tree scan, just assume it's unsafe */
glob->maxParallelHazard = PROPARALLEL_UNSAFE; parallelModeOK = false;
}+1.
In the current parallel_dml option patch. I put this check and some high-level check in a separate function called is_parallel_possible_for_modify.
- * PROPARALLEL_UNSAFE, PROPARALLEL_RESTRICTED, PROPARALLEL_SAFE + * Check at a high-level if parallel mode is able to be used for the specified + * table-modification statement. + * It's not possible in the following cases: + * + * 1) enable_parallel_dml is off + * 2) UPDATE or DELETE command + * 3) INSERT...ON CONFLICT...DO UPDATE + * 4) INSERT without SELECT on a relation + * 5) the reloption parallel_dml_enabled is not set for the target table + * + * (Note: we don't do in-depth parallel-safety checks here, we do only the + * cheaper tests that can quickly exclude obvious cases for which + * parallelism isn't supported, to avoid having to do further parallel-safety + * checks for these) */ +bool +is_parallel_possible_for_modify(Query *parse)And I put the function at earlier place like the following:
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && IsUnderPostmaster && (parse->commandType == CMD_SELECT || - (enable_parallel_dml && - IsModifySupportedInParallelMode(parse->commandType))) && + is_parallel_possible_for_modify(parse)) && !parse->hasModifyingCTE && max_parallel_workers_per_gather > 0 && !IsParallelWorker())If this looks good, maybe we can merge this change.
If I've understood correctly, you're suggesting to merge the
"is_parallel_possible_for_modify()" code from your parallel_dml patch
into the main parallel INSERT patch, right?
If so, I think that's a good idea, as it will help simplify both
patches (and then, if need be, we can still discuss where best to
place certain checks ...).
I'll post an update soon that includes that change (then the
parallel_dml patch will need to be rebased accordingly).
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):
Does any current Postgres code add partition OIDs to
PlannerGlobal.invalItems for a similar reason?
I would have thought that, for example, partitions with a default
column expression, using a function that is changed from SAFE to
UNSAFE, would suffer the same plancache issue (for current parallel
SELECT functionality) as we're talking about here - but so far I
haven't seen any code handling this.
(Currently invalItems seems to support PROCID and TYPEOID; relation
OIDs seem to be handled through a different mechanism)..
Can you elaborate on what you believe is required here, so that the
partition OID dependency is registered and the altered partition
results in the plan being invalidated?
Thanks in advance for any help you can provide here.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Feb 11, 2021 at 5:33 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):Does any current Postgres code add partition OIDs to
PlannerGlobal.invalItems for a similar reason?
I would have thought that, for example, partitions with a default
column expression, using a function that is changed from SAFE to
UNSAFE, would suffer the same plancache issue (for current parallel
SELECT functionality) as we're talking about here - but so far I
haven't seen any code handling this.(Currently invalItems seems to support PROCID and TYPEOID; relation
OIDs seem to be handled through a different mechanism)..
Can you elaborate on what you believe is required here, so that the
partition OID dependency is registered and the altered partition
results in the plan being invalidated?
Thanks in advance for any help you can provide here.
Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:
glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);
Can you confirm, is that what you were referring to?
(note that I've already updated the code to use
CreatePartitionDirectory() and PartitionDirectoryLookup())
Regards,
Greg Nancarrow
Fujitsu Australia
Posting an updated set of patches. Changes are based on feedback, as
detailed below:
[Amit Langote]
- Integrate max_parallel_hazard_for_modify() with max_parallel_hazard()
- Some function name changes
- Fix partition-related problems (to handle concurrent attachment of
partitions and altering of partitions, plan cache invalidation) and
added some tests for this.
(Method of fixing yet to be verified)
[Hou-san]
- Merge is_parallel_possible_for_modify() from the parallel_dml patch,
which helps in placement of some short-circuits of parallel-safety
checks
- Minor update to documentation for temp and foreign tables
[Greg]
- Temporary fix for query rewriter hasModifyingCTE bug (without
changing query rewriter code - note that v15 patch put fix in query
rewriter)
Hou-san: the parallel_dml patches will need slight rebasing.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v16-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v16-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 6fcbbe4d084510a78a57e3c2840ac7d05b26b7d0 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 11 Feb 2021 19:14:25 +1100
Subject: [PATCH v16 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++--
src/test/regress/expected/insert_parallel.out | 134 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 22 +--
3 files changed, 153 insertions(+), 86 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index ed5db202f9..1a56469aed 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -173,9 +173,9 @@ create table test_data1(like test_data);
explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -341,9 +341,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -391,9 +391,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -426,9 +426,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -462,9 +462,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -500,9 +500,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -540,9 +540,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -614,21 +614,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -657,7 +657,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -710,14 +710,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -766,7 +766,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -795,14 +795,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -962,7 +962,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -983,11 +983,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1028,7 +1028,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1085,13 +1085,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain execute q;
QUERY PLAN
--------------------------------------------------------------------------
- Insert on rp (cost=0.00..88.03 rows=0 width=0)
- -> Gather (cost=0.00..88.03 rows=57 width=4)
- Workers Planned: 4
+ Gather (cost=0.00..88.03 rows=57 width=0)
+ Workers Planned: 4
+ -> Insert on rp (cost=0.00..88.03 rows=0 width=0)
-> Parallel Seq Scan on foo (cost=0.00..88.03 rows=14 width=4)
Filter: ((a % 2) = 0)
(5 rows)
@@ -1118,9 +1118,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index bd955c71d8..6670615dc8 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -266,14 +266,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -281,7 +281,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -307,7 +307,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -323,7 +323,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -331,7 +331,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -415,7 +415,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -465,7 +465,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -511,7 +511,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
v16-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v16-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From f2abdf3ad0b45a58eff4090904c6a4a172ff4743 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 10 Feb 2021 14:50:40 +1100
Subject: [PATCH v16 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 28 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 511 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dd70ec2b21..4d2923a2fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index e8414d9310..2d1a6af3e2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -790,7 +790,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1527,7 +1528,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 5d90337498..bd7264bd89 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index a64d07a57c..caf884dff7 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -333,10 +336,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1805,7 +1809,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1853,6 +1970,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2293,96 +2411,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2401,6 +2456,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7569,7 +7631,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..595760b19f 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1050,6 +1053,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index d65099c94a..543bf1255c 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1170,7 +1170,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v16-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v16-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From a160f5eca3ce33c5d1121ed1d80b89f8f27692da Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 11 Feb 2021 19:24:36 +1100
Subject: [PATCH v16 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a TEMPORARY fix for a bug in the query rewriter.
Discussion: https://www.postgresql-archive.org/Bug-in-query-rewriter-hasModifyingCTE-not-getting-set-td6176917.html
It was found that for re-written queries with a modifying CTE, the
hasModifyingCTE flag was not getting set to true - resulting in a test failure
in the "with" tests when force_parallel_mode=regress was in effect, as it
allowed a CTE with a DELETE statement to be executed in parallel-mode.
This TEMPORARY fix (made in the planner) will need to be removed once the bug in
query rewriter is fixed.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 98 +++++-
src/backend/optimizer/util/clauses.c | 489 ++++++++++++++++++++++++++-
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 4 +-
6 files changed, 615 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..dd70ec2b21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index f4dd47acc7..e8414d9310 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1526,7 +1526,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index adf68d8790..a64d07a57c 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -246,7 +246,7 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *targetList,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
+static bool query_has_modifying_cte(Query *parse);
/*****************************************************************************
*
@@ -313,19 +313,30 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;
+ if (!parse->hasModifyingCTE)
+ {
+ /*
+ * FIXME
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this temporary fix must be removed.
+ */
+ parse->hasModifyingCTE = query_has_modifying_cte(parse);
+ }
+
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +345,15 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -7768,3 +7781,64 @@ group_by_has_partkey(RelOptInfo *input_rel,
return true;
}
+
+/*
+ * Determine if the specified query has a modifying-CTE.
+ *
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this function should be removed.
+ *
+ */
+static bool
+query_has_modifying_cte(Query *parsetree)
+{
+ bool hasModifyingCTE = false;
+
+ if (parsetree->querySource == QSRC_ORIGINAL)
+ {
+ /* Assume original queries have hasModifyingCTE set correctly */
+ if (parsetree->hasModifyingCTE)
+ hasModifyingCTE = true;
+ }
+
+ if (!hasModifyingCTE)
+ {
+ int rt_index;
+
+ /* Recursively check subqueries */
+ rt_index = 0;
+ while (rt_index < list_length(parsetree->rtable))
+ {
+ RangeTblEntry *rte;
+ ++rt_index;
+ rte = rt_fetch(rt_index, parsetree->rtable);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasModifyingCTE = query_has_modifying_cte(rte->subquery);
+ if (hasModifyingCTE)
+ break;
+ }
+ }
+
+ if (!hasModifyingCTE)
+ {
+ ListCell *lc;
+
+ /* Check for INSERT/UPDATE/DELETE CTEs */
+ foreach(lc, parsetree->cteList)
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ hasModifyingCTE = true;
+ break;
+ }
+ }
+ }
+ }
+
+ return hasModifyingCTE;
+}
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..de16a28639 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +170,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +566,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +609,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -757,6 +786,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +819,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +830,445 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_expr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+
+ /* Register the partition as a plan dependency. */
+ glob->relationOids =
+ lappend_oid(glob->relationOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (target_rel_index_expr_max_parallel_hazard(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_possible_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_possible_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ Assert(IsModifySupportedInParallelMode(parse->commandType));
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return false;
+
+ return true;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..e2f0fe3aa6 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,7 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
+extern bool is_parallel_possible_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v16-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v16-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From e828f2be0e98e3e98201fa52c0ce0d4634bc6338 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 11 Feb 2021 18:54:40 +1100
Subject: [PATCH v16 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1228 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 608 ++++++++
5 files changed, 1841 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..ed5db202f9
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1228 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain execute q;
+ QUERY PLAN
+--------------------------------------------------------------------------
+ Insert on rp (cost=0.00..88.03 rows=0 width=0)
+ -> Gather (cost=0.00..88.03 rows=57 width=4)
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo (cost=0.00..88.03 rows=14 width=4)
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain execute q;
+ QUERY PLAN
+------------------------------------------------------------
+ Insert on rp (cost=0.00..217.12 rows=0 width=0)
+ -> Seq Scan on foo (cost=0.00..217.12 rows=57 width=4)
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 12bb67e491..37dc86359b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 59b416fd80..4504af73cb 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..bd955c71d8
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,608 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain(costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain(costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain(costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain(costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain execute q;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
On Thu, Feb 11, 2021 at 11:17 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches. Changes are based on feedback, as
detailed below:
Oops, looks like I forgot "COSTS OFF" on some added EXPLAINs in the
tests, and it caused some test failures in the PostgreSQL Patch Tester
(cfbot).
Also, I think that perhaps the localized temporary fix included in the
patch for the hasModifyingCTE bug should be restricted to INSERT, even
though the bug actually exists for SELECT too.
Posting an updated set of patches to address these.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v17-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v17-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 4106f2563681012b37681a6701ac55a43979e55b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 12 Feb 2021 12:58:46 +1100
Subject: [PATCH v17 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++--
src/test/regress/expected/insert_parallel.out | 136 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 ++--
3 files changed, 155 insertions(+), 88 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 638eafd2d7..a5a309aff1 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -167,15 +167,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -341,9 +341,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -391,9 +391,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -426,9 +426,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -462,9 +462,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -500,9 +500,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -540,9 +540,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -614,21 +614,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -657,7 +657,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -710,14 +710,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -766,7 +766,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -795,14 +795,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -962,7 +962,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -983,11 +983,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1028,7 +1028,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1085,13 +1085,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1118,9 +1118,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index cb02ba89e4..9183032c07 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -122,7 +122,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -266,14 +266,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -281,7 +281,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -307,7 +307,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -323,7 +323,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -331,7 +331,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -415,7 +415,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -465,7 +465,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -511,7 +511,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
v17-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v17-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 8e78a3141c749bb5948ab7a862864268c1b53f27 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 12 Feb 2021 13:14:22 +1100
Subject: [PATCH v17 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a TEMPORARY fix for a bug in the query rewriter.
Discussion: https://www.postgresql-archive.org/Bug-in-query-rewriter-hasModifyingCTE-not-getting-set-td6176917.html
It was found that for re-written queries with a modifying CTE, the
hasModifyingCTE flag was not getting set to true - resulting in a test failure
in the "with" tests when force_parallel_mode=regress was in effect, as it
allowed a CTE with a DELETE statement to be executed in parallel-mode.
This TEMPORARY fix (made in the planner) will need to be removed once the bug in
query rewriter is fixed. Note that this fix is currently just done for INSERT,
but the problem actually exists for SELECT too.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 103 +++++-
src/backend/optimizer/util/clauses.c | 489 ++++++++++++++++++++++++++-
src/include/access/xact.h | 15 +
src/include/optimizer/clauses.h | 4 +-
6 files changed, 620 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index a2068e3fd4..dd70ec2b21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index adf68d8790..9f83e5e65f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -246,7 +246,7 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *targetList,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
+static bool query_has_modifying_cte(Query *parse);
/*****************************************************************************
*
@@ -313,19 +313,35 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !parse->hasModifyingCTE)
+ {
+ /*
+ * FIXME
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this temporary fix must be removed.
+ *
+ * Note that here we've made a fix for this problem only for a
+ * supported-in-parallel-mode table-modification statement (i.e.
+ * INSERT), but this bug exists for SELECT too.
+ */
+ parse->hasModifyingCTE = query_has_modifying_cte(parse);
+ }
+
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +350,15 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -7768,3 +7786,64 @@ group_by_has_partkey(RelOptInfo *input_rel,
return true;
}
+
+/*
+ * Determine if the specified query has a modifying-CTE.
+ *
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this function should be removed.
+ *
+ */
+static bool
+query_has_modifying_cte(Query *parsetree)
+{
+ bool hasModifyingCTE = false;
+
+ if (parsetree->querySource == QSRC_ORIGINAL)
+ {
+ /* Assume original queries have hasModifyingCTE set correctly */
+ if (parsetree->hasModifyingCTE)
+ hasModifyingCTE = true;
+ }
+
+ if (!hasModifyingCTE)
+ {
+ int rt_index;
+
+ /* Recursively check subqueries */
+ rt_index = 0;
+ while (rt_index < list_length(parsetree->rtable))
+ {
+ RangeTblEntry *rte;
+ ++rt_index;
+ rte = rt_fetch(rt_index, parsetree->rtable);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasModifyingCTE = query_has_modifying_cte(rte->subquery);
+ if (hasModifyingCTE)
+ break;
+ }
+ }
+
+ if (!hasModifyingCTE)
+ {
+ ListCell *lc;
+
+ /* Check for INSERT/UPDATE/DELETE CTEs */
+ foreach(lc, parsetree->cteList)
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ hasModifyingCTE = true;
+ break;
+ }
+ }
+ }
+ }
+
+ return hasModifyingCTE;
+}
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..de16a28639 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +170,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +566,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +609,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -757,6 +786,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +819,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +830,445 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_expr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+
+ /* Register the partition as a plan dependency. */
+ glob->relationOids =
+ lappend_oid(glob->relationOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (target_rel_index_expr_max_parallel_hazard(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only INSERT is currently supported, only
+ * command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are
+ * already being checked for parallel-safety in the
+ * max_parallel_hazard() scan of the query tree in standard_planner().
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Check if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_possible_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_possible_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ Assert(IsModifySupportedInParallelMode(parse->commandType));
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+ if (!hasSubQuery)
+ return false;
+
+ return true;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..e2f0fe3aa6 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,7 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
+extern bool is_parallel_possible_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v17-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v17-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 9649575d7e8bd2572cba638a460744615017f168 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 10 Feb 2021 14:50:40 +1100
Subject: [PATCH v17 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 28 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 511 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dd70ec2b21..4d2923a2fc 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9f83e5e65f..9e445f9f5d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -338,10 +341,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1810,7 +1814,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1858,6 +1975,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2298,96 +2416,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2406,6 +2461,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7574,7 +7636,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..595760b19f 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -252,6 +252,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -302,7 +303,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1050,6 +1053,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index b6a88ff76b..52a88245d7 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1174,7 +1174,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v17-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v17-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 1e0b7a4022a8be8a6d68204ed8c4043e5bb2dacb Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 12 Feb 2021 12:12:38 +1100
Subject: [PATCH v17 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1228 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 608 ++++++++
5 files changed, 1841 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..638eafd2d7
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1228 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 12bb67e491..37dc86359b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 59b416fd80..4504af73cb 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..cb02ba89e4
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,608 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
Hi,
For v17-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :
+ /* Assume original queries have hasModifyingCTE set correctly */
+ if (parsetree->hasModifyingCTE)
+ hasModifyingCTE = true;
Since hasModifyingCTE is false by the time the above is run, it can be
simplified as:
hasModifyingCTE = parsetree->hasModifyingCTE
+ if (!hasSubQuery)
+ return false;
+
+ return true;
The above can be simplified as:
return hasSubQuery;
Cheers
On Thu, Feb 11, 2021 at 7:02 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Show quoted text
On Thu, Feb 11, 2021 at 11:17 PM Greg Nancarrow <gregn4422@gmail.com>
wrote:Posting an updated set of patches. Changes are based on feedback, as
detailed below:Oops, looks like I forgot "COSTS OFF" on some added EXPLAINs in the
tests, and it caused some test failures in the PostgreSQL Patch Tester
(cfbot).
Also, I think that perhaps the localized temporary fix included in the
patch for the hasModifyingCTE bug should be restricted to INSERT, even
though the bug actually exists for SELECT too.
Posting an updated set of patches to address these.Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 12, 2021 at 2:33 PM Zhihong Yu <zyu@yugabyte.com> wrote:
For v17-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :
+ /* Assume original queries have hasModifyingCTE set correctly */ + if (parsetree->hasModifyingCTE) + hasModifyingCTE = true;Since hasModifyingCTE is false by the time the above is run, it can be simplified as:
hasModifyingCTE = parsetree->hasModifyingCTE
Actually, we should just return parsetree->hasModifyingCTE at this
point, because if it's false, we shouldn't need to continue the search
(as we're assuming it has been set correctly for QSRC_ORIGINAL case).
+ if (!hasSubQuery) + return false; + + return true;The above can be simplified as:
return hasSubQuery;
Yes, absolutely right, silly miss on that one!
Thanks.
This was only ever meant to be a temporary fix for this bug that
affects this patch.
Regards,
Greg Nancarrow
Fujitsu Australia
Greg:
bq. we should just return parsetree->hasModifyingCTE at this point,
Maybe you can clarify a bit.
The if (parsetree->hasModifyingCTE) check is followed by if
(!hasModifyingCTE).
When parsetree->hasModifyingCTE is false, !hasModifyingCTE would be true,
resulting in the execution of the if (!hasModifyingCTE) block.
In your reply, did you mean that the if (!hasModifyingCTE) block is no
longer needed ? (I guess not)
Cheers
On Thu, Feb 11, 2021 at 8:14 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Show quoted text
On Fri, Feb 12, 2021 at 2:33 PM Zhihong Yu <zyu@yugabyte.com> wrote:
For v17-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patch :
+ /* Assume original queries have hasModifyingCTE set correctly */ + if (parsetree->hasModifyingCTE) + hasModifyingCTE = true;Since hasModifyingCTE is false by the time the above is run, it can be
simplified as:
hasModifyingCTE = parsetree->hasModifyingCTE
Actually, we should just return parsetree->hasModifyingCTE at this
point, because if it's false, we shouldn't need to continue the search
(as we're assuming it has been set correctly for QSRC_ORIGINAL case).+ if (!hasSubQuery) + return false; + + return true;The above can be simplified as:
return hasSubQuery;Yes, absolutely right, silly miss on that one!
Thanks.This was only ever meant to be a temporary fix for this bug that
affects this patch.Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 12, 2021 at 3:21 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Greg:
bq. we should just return parsetree->hasModifyingCTE at this point,Maybe you can clarify a bit.
The if (parsetree->hasModifyingCTE) check is followed by if (!hasModifyingCTE).
When parsetree->hasModifyingCTE is false, !hasModifyingCTE would be true, resulting in the execution of the if (!hasModifyingCTE) block.In your reply, did you mean that the if (!hasModifyingCTE) block is no longer needed ? (I guess not)
Sorry for not making it clear. What I meant was that instead of:
if (parsetree->querySource == QSRC_ORIGINAL)
{
/* Assume original queries have hasModifyingCTE set correctly */
if (parsetree->hasModifyingCTE)
hasModifyingCTE = true;
}
I thought I should be able to use the following (it the setting for
QSRC_ORIGINAL can really be trusted):
if (parsetree->querySource == QSRC_ORIGINAL)
{
/* Assume original queries have hasModifyingCTE set correctly */
return parsetree->hasModifyingCTE;
}
(and then the "if (!hasModifyingCTE)" test on the code following
immediately below it can be removed)
BUT - after testing that change, the problem test case (in the "with"
tests) STILL fails.
I then checked if hasModifyingCTE is always false in the QSRC_ORIGINAL
case (by adding an Assert), and it always is false.
So actually, there is no point in having the "if
(parsetree->querySource == QSRC_ORIGINAL)" code block - even the so
called "original" query doesn't maintain the setting correctly (even
though the actual original query sent into the query rewriter does).
And also then the "if (!hasModifyingCTE)" test on the code following
immediately below it can be removed.
Regards,
Greg Nancarrow
Fujitsu Australia
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
What would the result look like if you turn off
parallel_leader_participation? If the leader is freed from
reading/writing the table and index, the index page splits and
internal lock contention may decrease enough to recover part of the loss.https://www.postgresql.org/docs/devel/parallel-plans.html
"In a parallel bitmap heap scan, one process is chosen as the leader.
That process performs a scan of one or more indexes and builds a
bitmap indicating which table blocks need to be visited. These blocks
are then divided among the cooperating processes as in a parallel
sequential scan. In other words, the heap scan is performed in
parallel, but the underlying index scan is not."If I disable parallel_leader_participation.
For max_parallel_workers_per_gather = 4, It still have performance
degradation.For max_parallel_workers_per_gather = 2, the performance degradation will
not happen in most of the case.
There is sometimes a noise(performance degradation), but most of
result(about 80%) is good.
Thank you. The results indicate that it depends on the degree of parallelism whether the gain from parallelism outweighs the loss of parallel insert operations, at least in the bitmap scan case.
But can we conclude that this is limited to bitmap scan? Even if that's the case, the planner does not have information about insert operation to choose other plans like serial execution or parallel sequential scan. Should we encourage the user in the manual to tune parameters and find the fastest plan?
Regards
Takayuki Tsunakawa
From: Hou, Zhijie/侯 志杰 <houzj.fnst@cn.fujitsu.com>
If we diable bitmapscan, the performance degradation seems will not happen.
Yes, but that's because the hundreds of times slower sequential scan hides the insert time. Furthermore, as an aside, Worker 3 does much of the work in the parallel sequential scan + parallel insert case, while the load is well balanced in the parallel bitmap scan + parallel insert case.
Oracle and SQL Server executes parallel DML by holding an exclusive lock on the target table. They might use some special path for parallel DML to mitigate contention.
[serial bitmap scan + serial insert]
-> Bitmap Heap Scan on public.x (cost=3272.20..3652841.26 rows=79918 width=8) (actual time=8.096..41.005 rows=129999 loops=1)
...
Execution Time: 360.547 ms
[parallel bitmap scan + parallel insert]
-> Parallel Bitmap Heap Scan on public.x (cost=3272.20..1260119.35 rows=19980 width=8) (actual time=5.832..14.787 rows=26000 loops=5)
...
Execution Time: 382.776 ms
[serial sequential scan + serial insert]
-> Seq Scan on public.x (cost=0.00..5081085.52 rows=81338 width=8) (actual time=0.010..18997.317 rows=129999 loops=1)
...
Execution Time: 19311.700 ms
[parallel sequential scan + parallel insert]
-> Parallel Seq Scan on public.x (cost=0.00..2081082.88 rows=20334 width=8) (actual time=4001.641..5287.248 rows=32500 loops=4)
...
Execution Time: 5488.493 ms
Regards
Takayuki Tsunakawa
Greg:
Thanks for more debugging.
Cheers
On Thu, Feb 11, 2021 at 9:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Show quoted text
On Fri, Feb 12, 2021 at 3:21 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Greg:
bq. we should just return parsetree->hasModifyingCTE at this point,Maybe you can clarify a bit.
The if (parsetree->hasModifyingCTE) check is followed by if(!hasModifyingCTE).
When parsetree->hasModifyingCTE is false, !hasModifyingCTE would be
true, resulting in the execution of the if (!hasModifyingCTE) block.
In your reply, did you mean that the if (!hasModifyingCTE) block is no
longer needed ? (I guess not)
Sorry for not making it clear. What I meant was that instead of:
if (parsetree->querySource == QSRC_ORIGINAL)
{
/* Assume original queries have hasModifyingCTE set correctly */
if (parsetree->hasModifyingCTE)
hasModifyingCTE = true;
}I thought I should be able to use the following (it the setting for
QSRC_ORIGINAL can really be trusted):if (parsetree->querySource == QSRC_ORIGINAL)
{
/* Assume original queries have hasModifyingCTE set correctly */
return parsetree->hasModifyingCTE;
}(and then the "if (!hasModifyingCTE)" test on the code following
immediately below it can be removed)BUT - after testing that change, the problem test case (in the "with"
tests) STILL fails.
I then checked if hasModifyingCTE is always false in the QSRC_ORIGINAL
case (by adding an Assert), and it always is false.
So actually, there is no point in having the "if
(parsetree->querySource == QSRC_ORIGINAL)" code block - even the so
called "original" query doesn't maintain the setting correctly (even
though the actual original query sent into the query rewriter does).
And also then the "if (!hasModifyingCTE)" test on the code following
immediately below it can be removed.Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 12, 2021 at 5:30 PM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
If I disable parallel_leader_participation.
For max_parallel_workers_per_gather = 4, It still have performance
degradation.For max_parallel_workers_per_gather = 2, the performance degradation will
not happen in most of the case.
There is sometimes a noise(performance degradation), but most of
result(about 80%) is good.Thank you. The results indicate that it depends on the degree of parallelism whether the gain from parallelism outweighs the loss of parallel insert operations, at least in the bitmap scan case.
That seems to be the pattern for this particular query, but I think
we'd need to test a variety to determine if that's always the case.
But can we conclude that this is limited to bitmap scan? Even if that's the case, the planner does not have information about insert operation to choose other plans like serial execution or parallel sequential scan. Should we encourage the user in the manual to tune parameters and find the fastest plan?
It's all based on path costs, so we need to analyze and compare the
costing calculations done in this particular case against other cases,
and the values of the various parameters (costsize.c).
It's not difficult to determine for a parallel ModifyTablePath if it
has a BitmapHeapPath subpath - perhaps total_cost needs adjustment
(increase) for this case - and that will influence the planner to
choose a cheaper path. I was able to easily test the effect of doing
this, in the debugger - by increasing total_cost in cost_modifytable()
for the parallel bitmap heap scan case, the planner then chose a
serial Insert + bitmap heap scan, because it then had a cheaper cost.
Of course we need to better understand the problem and observed
patters in order to get a better feel of how those costs should be
adjusted.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Feb 11, 2021 at 4:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Feb 11, 2021 at 5:33 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):Does any current Postgres code add partition OIDs to
PlannerGlobal.invalItems for a similar reason?
Currently, the planner opens partitions only for SELECT queries and
also adds them to the query's range table. And because they are added
to the range table, their OIDs do get added to
PlannerGlobal.relationOids (not invalItems, sorry!) by way of
CompleteCachedPlan() calling extract_query_dependencies(), which looks
at Query.rtable to decide which tables/partitions to add.
I would have thought that, for example, partitions with a default
column expression, using a function that is changed from SAFE to
UNSAFE, would suffer the same plancache issue (for current parallel
SELECT functionality) as we're talking about here - but so far I
haven't seen any code handling this.
AFAIK, default column expressions don't affect plans for SELECT
queries. OTOH, consider a check constraint expression as an example.
The planner may use one to exclude a partition from the plan with its
constraint exclusion algorithm (separate from "partition pruning").
If the check constraint is dropped, any cached plans that used it will
be invalidated.
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
alter table rp1 add constraint chk check (a >= -5);
set constraint_exclusion to on;
-- forces using a cached plan
set plan_cache_mode to force_generic_plan ;
prepare q as select * from rp where a < -5;
-- planner excluded rp1 because of the contradictory constraint
explain execute q;
QUERY PLAN
------------------------------------------
Result (cost=0.00..0.00 rows=0 width=0)
One-Time Filter: false
(2 rows)
-- constraint dropped, plancache inval hook invoked
alter table rp1 drop constraint chk ;
-- old plan invalidated, new one made
explain execute q;
QUERY PLAN
---------------------------------------------------------
Seq Scan on rp1 rp (cost=0.00..41.88 rows=850 width=4)
Filter: (a < '-5'::integer)
(2 rows)
(Currently invalItems seems to support PROCID and TYPEOID; relation
OIDs seem to be handled through a different mechanism)..Can you elaborate on what you believe is required here, so that the
partition OID dependency is registered and the altered partition
results in the plan being invalidated?
Thanks in advance for any help you can provide here.Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);Can you confirm, is that what you were referring to?
Right. I had mistakenly mentioned PlannerGlobal.invalItems, sorry.
Although it gets the job done, I'm not sure if manipulating
relationOids from max_parallel_hazard() or its subroutines is okay,
but I will let the committer decide that. As I mentioned above, the
person who designed this decided for some reason that it is
extract_query_dependencies()'s job to populate
PlannerGlobal.relationOids/invalItems.
(note that I've already updated the code to use
CreatePartitionDirectory() and PartitionDirectoryLookup())
I will check your v16 to check if that indeed does the intended thing.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Sat, Feb 13, 2021 at 12:17 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Thu, Feb 11, 2021 at 4:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Feb 11, 2021 at 5:33 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 9, 2021 at 1:04 AM Amit Langote <amitlangote09@gmail.com> wrote:
* I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties. See this (tested with all
patches applied!):Does any current Postgres code add partition OIDs to
PlannerGlobal.invalItems for a similar reason?Currently, the planner opens partitions only for SELECT queries and
also adds them to the query's range table. And because they are added
to the range table, their OIDs do get added to
PlannerGlobal.relationOids (not invalItems, sorry!) by way of
CompleteCachedPlan() calling extract_query_dependencies(), which looks
at Query.rtable to decide which tables/partitions to add.I would have thought that, for example, partitions with a default
column expression, using a function that is changed from SAFE to
UNSAFE, would suffer the same plancache issue (for current parallel
SELECT functionality) as we're talking about here - but so far I
haven't seen any code handling this.AFAIK, default column expressions don't affect plans for SELECT
queries. OTOH, consider a check constraint expression as an example.
The planner may use one to exclude a partition from the plan with its
constraint exclusion algorithm (separate from "partition pruning").
If the check constraint is dropped, any cached plans that used it will
be invalidated.
Sorry, I got that wrong, default column expressions are relevant to
INSERT, not SELECT.
Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);Can you confirm, is that what you were referring to?
Right. I had mistakenly mentioned PlannerGlobal.invalItems, sorry.
Although it gets the job done, I'm not sure if manipulating
relationOids from max_parallel_hazard() or its subroutines is okay,
but I will let the committer decide that. As I mentioned above, the
person who designed this decided for some reason that it is
extract_query_dependencies()'s job to populate
PlannerGlobal.relationOids/invalItems.
Yes, it doesn't really seem right doing it within max_parallel_hazard().
I tried doing it in extract_query_dependencies() instead - see
attached patch - and it seems to work, but I'm not sure if there might
be any unintended side-effects.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
setrefs.patchapplication/octet-stream; name=setrefs.patchDownload
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 595760b19f..6b121fc3a1 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,9 @@
*/
#include "postgres.h"
+#include "access/table.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +26,7 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -3018,6 +3021,9 @@ extract_query_dependencies(Node *query,
*relationOids = glob.relationOids;
*invalItems = glob.invalItems;
*hasRowSecurity = glob.dependsOnRole;
+
+ if (glob.partition_directory != NULL)
+ DestroyPartitionDirectory(glob.partition_directory);
}
/*
@@ -3060,8 +3066,36 @@ extract_query_dependencies_walker(Node *node, PlannerInfo *context)
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);
if (rte->rtekind == RTE_RELATION)
- context->glob->relationOids =
- lappend_oid(context->glob->relationOids, rte->relid);
+ {
+ PlannerGlobal *glob;
+
+ glob = context->glob;
+ glob->relationOids =
+ lappend_oid(glob->relationOids, rte->relid);
+ if (query->commandType == CMD_INSERT &&
+ rte->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ Relation rel;
+ PartitionDesc pdesc;
+
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ rel = table_open(rte->relid, NoLock);
+ pdesc =
+ PartitionDirectoryLookup(glob->partition_directory,
+ rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ glob->relationOids =
+ lappend_oid(glob->relationOids, pdesc->oids[i]);
+ }
+ table_close(rel, NoLock);
+ }
+ }
else if (rte->rtekind == RTE_NAMEDTUPLESTORE &&
OidIsValid(rte->relid))
context->glob->relationOids =
On Mon, Feb 8, 2021 at 8:13 PM Hou, Zhijie <houzj.fnst@cn.fujitsu.com> wrote:
Did it actually use a parallel plan in your testing?
When I ran these tests with the Parallel INSERT patch applied, it did
not naturally choose a parallel plan for any of these cases.Yes, these cases pick parallel plan naturally on my test environment.
postgres=# explain verbose insert into testscan select a from x where
a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
----------------------------------------------------------------------
-----------------------------
Gather (cost=4346.89..1281204.64 rows=81372 width=0)
Workers Planned: 4
-> Insert on public.testscan (cost=3346.89..1272067.44 rows=0
width=0)
-> Parallel Bitmap Heap Scan on public.x1
(cost=3346.89..1272067.44 rows=20343 width=8)
Output: x1.a, NULL::integer
Recheck Cond: ((x1.a < 80000) OR (x1.a > 199900000))
Filter: ((x1.a < 80000) OR (((x1.a % 2) = 0) AND (x1.a >
199900000)))
-> BitmapOr (cost=3346.89..3346.89 rows=178808
width=0)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1495.19 rows=80883 width=0)
Index Cond: (x1.a < 80000)
-> Bitmap Index Scan on x1_a_idx
(cost=0.00..1811.01 rows=97925 width=0)
Index Cond: (x1.a > 199900000)PSA is my postgresql.conf file, maybe you can have a look. Besides, I didn't
do any parameters tuning in my test session.I reproduced this on my machine.
I think we'd better do "analyze" before insert which helps reproduce this easier.
Like:-----
analyze;
explain analyze verbose insert into testscan select a from x where a<80000 or (a%2=0 and a>199900000);
-----
Thanks, I tried test_bimap.sql in my own environment, and added
"analyze", and I also found it naturally chose a parallel INSERT with
parallel bitmap heap scan for each of these cases.
However, I didn't see any performance degradation when compared
against serial INSERT with bitmap heap scan.
The parallel plan in these cases seems to run a bit faster.
(Note that I'm using a release build of Postgres, and using default
postgresql.conf)
test=# set max_parallel_workers_per_gather=4;
SET
test=# explain analyze verbose insert into testscan select a from x
where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------------
Gather (cost=4193.29..1255440.94 rows=74267 width=0) (actual
time=210.587..212.135 rows=0 loops=1)
Workers Planned: 4
Workers Launched: 4
-> Insert on public.testscan (cost=3193.29..1247014.24 rows=0
width=0) (actual time=195.296..195.298 rows=0 loops=5)
Worker 0: actual time=189.512..189.514 rows=0 loops=1
Worker 1: actual time=194.843..194.844 rows=0 loops=1
Worker 2: actual time=193.986..193.988 rows=0 loops=1
Worker 3: actual time=188.035..188.037 rows=0 loops=1
-> Parallel Bitmap Heap Scan on public.x
(cost=3193.29..1247014.24 rows=18567 width=8) (actual
time=7.992..25.837 row
s=26000 loops=5)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a >
199900000)))
Rows Removed by Filter: 10000
Heap Blocks: exact=261
Worker 0: actual time=1.473..14.458 rows=22465 loops=1
Worker 1: actual time=7.370..31.359 rows=30525 loops=1
Worker 2: actual time=8.765..19.838 rows=18549 loops=1
Worker 3: actual time=0.279..17.269 rows=23864 loops=1
-> BitmapOr (cost=3193.29..3193.29 rows=170535
width=0) (actual time=21.775..21.777 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1365.94 rows=73783 width=0) (actual time=11.961..11.961
rows=
79999 loops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1790.21 rows=96752 width=0) (actual time=9.809..9.809
rows=10
0000 loops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.276 ms
Execution Time: 212.189 ms
(25 rows)
test=# truncate testscan;
TRUNCATE TABLE
test=# set max_parallel_workers_per_gather=0;
SET
test=# explain analyze verbose insert into testscan select a from x
where a<80000 or (a%2=0 and a>199900000);
QUERY
PLAN
--------------------------------------------------------------------------------------------------------------------------------
Insert on public.testscan (cost=3193.29..3625636.35 rows=0 width=0)
(actual time=241.222..241.224 rows=0 loops=1)
-> Bitmap Heap Scan on public.x (cost=3193.29..3625636.35
rows=74267 width=8) (actual time=16.945..92.392 rows=129999 loops
=1)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199900000)))
Rows Removed by Filter: 50000
Heap Blocks: exact=975
-> BitmapOr (cost=3193.29..3193.29 rows=170535 width=0)
(actual time=16.735..16.736 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1365.94
rows=73783 width=0) (actual time=9.222..9.223 rows=79999 lo
ops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1790.21
rows=96752 width=0) (actual time=7.511..7.511 rows=100000 l
oops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.205 ms
Execution Time: 241.274 ms
(14 rows)
============
test=# set max_parallel_workers_per_gather=4;
SET
test=# explain analyze verbose insert into testscan_pk select a from x
where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------------
Gather (cost=4193.29..1255440.94 rows=74267 width=0) (actual
time=572.242..573.683 rows=0 loops=1)
Workers Planned: 4
Workers Launched: 4
-> Insert on public.testscan_pk (cost=3193.29..1247014.24 rows=0
width=0) (actual time=566.303..566.308 rows=0 loops=5)
Worker 0: actual time=566.756..566.757 rows=0 loops=1
Worker 1: actual time=564.778..564.779 rows=0 loops=1
Worker 2: actual time=564.402..564.419 rows=0 loops=1
Worker 3: actual time=563.748..563.749 rows=0 loops=1
-> Parallel Bitmap Heap Scan on public.x
(cost=3193.29..1247014.24 rows=18567 width=8) (actual
time=16.479..37.327 ro
ws=26000 loops=5)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a >
199900000)))
Rows Removed by Filter: 10000
Heap Blocks: exact=204
Worker 0: actual time=17.358..36.895 rows=24233 loops=1
Worker 1: actual time=12.711..33.538 rows=25616 loops=1
Worker 2: actual time=15.671..35.701 rows=24831 loops=1
Worker 3: actual time=17.656..39.310 rows=26645 loops=1
-> BitmapOr (cost=3193.29..3193.29 rows=170535
width=0) (actual time=18.541..18.542 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1365.94 rows=73783 width=0) (actual time=8.549..8.549
rows=79
999 loops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1790.21 rows=96752 width=0) (actual time=9.990..9.990
rows=10
0000 loops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.240 ms
Execution Time: 573.733 ms
(25 rows)
test=# set max_parallel_workers_per_gather=0;
SET
test=# truncate testscan_pk;
TRUNCATE TABLE
test=# explain analyze verbose insert into testscan_pk select a from x
where a<80000 or (a%2=0 and a>199900000);
QUERY
PLAN
--------------------------------------------------------------------------------------------------------------------------------
Insert on public.testscan_pk (cost=3193.29..3625636.35 rows=0
width=0) (actual time=598.997..598.998 rows=0 loops=1)
-> Bitmap Heap Scan on public.x (cost=3193.29..3625636.35
rows=74267 width=8) (actual time=20.153..96.858 rows=129999 loops
=1)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199900000)))
Rows Removed by Filter: 50000
Heap Blocks: exact=975
-> BitmapOr (cost=3193.29..3193.29 rows=170535 width=0)
(actual time=19.840..19.841 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1365.94
rows=73783 width=0) (actual time=9.276..9.276 rows=79999 lo
ops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1790.21
rows=96752 width=0) (actual time=10.562..10.562 rows=100000
loops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.204 ms
Execution Time: 599.098 ms
(14 rows)
============
test=# set max_parallel_workers_per_gather=4;
SET
test=# explain analyze verbose insert into testscan_index select a
from x where a<80000 or (a%2=0 and a>199900000);
QUERY PLAN
--------------------------------------------------------------------------------------------------------------------------------
Gather (cost=4193.29..1255440.94 rows=74267 width=0) (actual
time=560.460..562.386 rows=0 loops=1)
Workers Planned: 4
Workers Launched: 4
-> Insert on public.testscan_index (cost=3193.29..1247014.24
rows=0 width=0) (actual time=553.434..553.435 rows=0 loops=5)
Worker 0: actual time=548.751..548.752 rows=0 loops=1
Worker 1: actual time=552.008..552.009 rows=0 loops=1
Worker 2: actual time=553.094..553.095 rows=0 loops=1
Worker 3: actual time=553.389..553.390 rows=0 loops=1
-> Parallel Bitmap Heap Scan on public.x
(cost=3193.29..1247014.24 rows=18567 width=8) (actual
time=13.759..34.487 ro
ws=26000 loops=5)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a >
199900000)))
Rows Removed by Filter: 10000
Heap Blocks: exact=183
Worker 0: actual time=8.698..29.924 rows=26173 loops=1
Worker 1: actual time=12.865..33.889 rows=27421 loops=1
Worker 2: actual time=13.088..32.823 rows=24591 loops=1
Worker 3: actual time=14.075..36.349 rows=26571 loops=1
-> BitmapOr (cost=3193.29..3193.29 rows=170535
width=0) (actual time=19.356..19.357 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1365.94 rows=73783 width=0) (actual time=10.330..10.330
rows=
79999 loops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx
(cost=0.00..1790.21 rows=96752 width=0) (actual time=9.024..9.024
rows=10
0000 loops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.219 ms
Execution Time: 562.442 ms
(25 rows)
test=# set max_parallel_workers_per_gather=0;
SET
test=# truncate testscan_index;
TRUNCATE TABLE
test=# explain analyze verbose insert into testscan_index select a
from x where a<80000 or (a%2=0 and a>199900000);
QUERY
PLAN
--------------------------------------------------------------------------------------------------------------------------------
Insert on public.testscan_index (cost=3193.29..3625636.35 rows=0
width=0) (actual time=607.619..607.621 rows=0 loops=1)
-> Bitmap Heap Scan on public.x (cost=3193.29..3625636.35
rows=74267 width=8) (actual time=21.001..96.283 rows=129999 loops
=1)
Output: x.a, NULL::integer
Recheck Cond: ((x.a < 80000) OR (x.a > 199900000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199900000)))
Rows Removed by Filter: 50000
Heap Blocks: exact=975
-> BitmapOr (cost=3193.29..3193.29 rows=170535 width=0)
(actual time=20.690..20.691 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1365.94
rows=73783 width=0) (actual time=9.097..9.097 rows=79999 lo
ops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1790.21
rows=96752 width=0) (actual time=11.591..11.591 rows=100000
loops=1)
Index Cond: (x.a > 199900000)
Planning Time: 0.205 ms
Execution Time: 607.734 ms
(14 rows)
Even when I changed the queries to return more rows from the scan, to
the point where it chose not to use a parallel INSERT bitmap heap scan
(in favour of parallel seq scan), and then forced it to by disabling
seqscan, I found that it was still at least as fast as serial INSERT
with bitmap heap scan.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Feb 15, 2021 at 4:39 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Feb 13, 2021 at 12:17 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Thu, Feb 11, 2021 at 4:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);Can you confirm, is that what you were referring to?
Right. I had mistakenly mentioned PlannerGlobal.invalItems, sorry.
Although it gets the job done, I'm not sure if manipulating
relationOids from max_parallel_hazard() or its subroutines is okay,
but I will let the committer decide that. As I mentioned above, the
person who designed this decided for some reason that it is
extract_query_dependencies()'s job to populate
PlannerGlobal.relationOids/invalItems.Yes, it doesn't really seem right doing it within max_parallel_hazard().
I tried doing it in extract_query_dependencies() instead - see
attached patch - and it seems to work, but I'm not sure if there might
be any unintended side-effects.
One issue I see with the patch is that it fails to consider
multi-level partitioning, because it's looking up partitions only in
the target table's PartitionDesc and no other.
@@ -3060,8 +3066,36 @@ extract_query_dependencies_walker(Node *node,
PlannerInfo *context)
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);
if (rte->rtekind == RTE_RELATION)
- context->glob->relationOids =
- lappend_oid(context->glob->relationOids, rte->relid);
+ {
+ PlannerGlobal *glob;
+
+ glob = context->glob;
+ glob->relationOids =
+ lappend_oid(glob->relationOids, rte->relid);
+ if (query->commandType == CMD_INSERT &&
+ rte->relkind == RELKIND_PARTITIONED_TABLE)
The RTE whose relkind is being checked here may not be the INSERT
target relation's RTE, even though that's perhaps always true today.
So, I suggest to pull the new block out of the loop over rtable and
perform its deeds on the result RTE explicitly fetched using
rt_fetch(), preferably using a separate recursive function. I'm
thinking something like the attached revised version.
--
Amit Langote
EDB: http://www.enterprisedb.com
Attachments:
setrefs-v2.patchapplication/octet-stream; name=setrefs-v2.patchDownload
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 595760b19f..d0adced270 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,9 @@
*/
#include "postgres.h"
+#include "access/table.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +26,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -3018,6 +3022,42 @@ extract_query_dependencies(Node *query,
*relationOids = glob.relationOids;
*invalItems = glob.invalItems;
*hasRowSecurity = glob.dependsOnRole;
+
+ if (glob.partition_directory != NULL)
+ DestroyPartitionDirectory(glob.partition_directory);
+}
+
+/*
+ * Recursively adds a table's partitions' OIDs to relationOids.
+ */
+static void
+add_target_partition_oids_recurse(Oid relid, int lockmode,
+ PlannerGlobal *glob)
+{
+ Relation rel;
+ PartitionDesc pdesc;
+ int i;
+
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ rel = table_open(relid, lockmode);
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ glob->relationOids = lappend_oid(glob->relationOids, pdesc->oids[i]);
+
+ /*
+ * If the partition may have its own partitions, recurse to add them
+ * too.
+ */
+ if (!pdesc->is_leaf[i])
+ add_target_partition_oids_recurse(pdesc->oids[i], lockmode, glob);
+ }
+
+ table_close(rel, NoLock);
}
/*
@@ -3069,6 +3109,17 @@ extract_query_dependencies_walker(Node *node, PlannerInfo *context)
rte->relid);
}
+ if (query->commandType == CMD_INSERT)
+ {
+ RangeTblEntry *result_rte = rt_fetch(query->resultRelation,
+ query->rtable);
+
+ if (result_rte->relkind == RELKIND_PARTITIONED_TABLE)
+ add_target_partition_oids_recurse(result_rte->relid,
+ result_rte->rellockmode,
+ context->glob);
+ }
+
/* And recurse into the query's subexpressions */
return query_tree_walker(query, extract_query_dependencies_walker,
(void *) context, 0);
On Wed, Feb 17, 2021 at 12:19 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Mon, Feb 15, 2021 at 4:39 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Feb 13, 2021 at 12:17 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Thu, Feb 11, 2021 at 4:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);Can you confirm, is that what you were referring to?
Right. I had mistakenly mentioned PlannerGlobal.invalItems, sorry.
Although it gets the job done, I'm not sure if manipulating
relationOids from max_parallel_hazard() or its subroutines is okay,
but I will let the committer decide that. As I mentioned above, the
person who designed this decided for some reason that it is
extract_query_dependencies()'s job to populate
PlannerGlobal.relationOids/invalItems.Yes, it doesn't really seem right doing it within max_parallel_hazard().
I tried doing it in extract_query_dependencies() instead - see
attached patch - and it seems to work, but I'm not sure if there might
be any unintended side-effects.One issue I see with the patch is that it fails to consider
multi-level partitioning, because it's looking up partitions only in
the target table's PartitionDesc and no other.@@ -3060,8 +3066,36 @@ extract_query_dependencies_walker(Node *node,
PlannerInfo *context)
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);if (rte->rtekind == RTE_RELATION) - context->glob->relationOids = - lappend_oid(context->glob->relationOids, rte->relid); + { + PlannerGlobal *glob; + + glob = context->glob; + glob->relationOids = + lappend_oid(glob->relationOids, rte->relid); + if (query->commandType == CMD_INSERT && + rte->relkind == RELKIND_PARTITIONED_TABLE)The RTE whose relkind is being checked here may not be the INSERT
target relation's RTE, even though that's perhaps always true today.
So, I suggest to pull the new block out of the loop over rtable and
perform its deeds on the result RTE explicitly fetched using
rt_fetch(), preferably using a separate recursive function. I'm
thinking something like the attached revised version.
Thanks. Yes, I'd forgotten about the fact a partition may itself be
partitioned, so it needs to be recursive (like in the parallel-safety
checks).
Your revised version seems OK, though I do have a concern:
Is the use of "table_close(rel, NoLock)'' intentional? That will keep
the lock (lockmode) until end-of-transaction.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 17, 2021 at 10:44 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 17, 2021 at 12:19 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Mon, Feb 15, 2021 at 4:39 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Sat, Feb 13, 2021 at 12:17 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Thu, Feb 11, 2021 at 4:43 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Actually, I tried adding the following in the loop that checks the
parallel-safety of each partition and it seemed to work:glob->relationOids =
lappend_oid(glob->relationOids, pdesc->oids[i]);Can you confirm, is that what you were referring to?
Right. I had mistakenly mentioned PlannerGlobal.invalItems, sorry.
Although it gets the job done, I'm not sure if manipulating
relationOids from max_parallel_hazard() or its subroutines is okay,
but I will let the committer decide that. As I mentioned above, the
person who designed this decided for some reason that it is
extract_query_dependencies()'s job to populate
PlannerGlobal.relationOids/invalItems.Yes, it doesn't really seem right doing it within max_parallel_hazard().
I tried doing it in extract_query_dependencies() instead - see
attached patch - and it seems to work, but I'm not sure if there might
be any unintended side-effects.One issue I see with the patch is that it fails to consider
multi-level partitioning, because it's looking up partitions only in
the target table's PartitionDesc and no other.@@ -3060,8 +3066,36 @@ extract_query_dependencies_walker(Node *node,
PlannerInfo *context)
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc);if (rte->rtekind == RTE_RELATION) - context->glob->relationOids = - lappend_oid(context->glob->relationOids, rte->relid); + { + PlannerGlobal *glob; + + glob = context->glob; + glob->relationOids = + lappend_oid(glob->relationOids, rte->relid); + if (query->commandType == CMD_INSERT && + rte->relkind == RELKIND_PARTITIONED_TABLE)The RTE whose relkind is being checked here may not be the INSERT
target relation's RTE, even though that's perhaps always true today.
So, I suggest to pull the new block out of the loop over rtable and
perform its deeds on the result RTE explicitly fetched using
rt_fetch(), preferably using a separate recursive function. I'm
thinking something like the attached revised version.Thanks. Yes, I'd forgotten about the fact a partition may itself be
partitioned, so it needs to be recursive (like in the parallel-safety
checks).
Your revised version seems OK, though I do have a concern:
Is the use of "table_close(rel, NoLock)'' intentional? That will keep
the lock (lockmode) until end-of-transaction.
I think we always keep any locks on relations that are involved in a
plan until end-of-transaction. What if a partition is changed in an
unsafe manner between being considered safe for parallel insertion and
actually performing the parallel insert?
BTW, I just noticed that exctract_query_dependencies() runs on a
rewritten, but not-yet-planned query tree, that is, I didn't know that
extract_query_dependencies() only populates the CachedPlanSource's
relationOids and not CachedPlan's. The former is only for tracking
the dependencies of an unplanned Query, so partitions should never be
added to it. Instead, they should be added to
PlannedStmt.relationOids (note PlannedStmt belongs to CachedPlan),
which is kind of what your earlier patch did. Needless to say,
PlanCacheRelCallback checks both CachedPlanSource.relationOids and
PlannedStmt.relationOids, so if it receives a message about a
partition, its OID is matched from the latter.
All that is to say that we should move our code to add partition OIDs
as plan dependencies to somewhere in set_plan_references(), which
otherwise populates PlannedStmt.relationOids. I updated the patch to
do that. It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.
--
Amit Langote
EDB: http://www.enterprisedb.com
Attachments:
setrefs-v3.patchapplication/octet-stream; name=setrefs-v3.patchDownload
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 595760b19f..0a3fe07049 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,9 @@
*/
#include "postgres.h"
+#include "access/table.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +26,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -108,6 +112,8 @@ static void add_rtes_to_flat_rtable(PlannerInfo *root, bool recursing);
static void flatten_unplanned_rtes(PlannerGlobal *glob, RangeTblEntry *rte);
static bool flatten_rtes_walker(Node *node, PlannerGlobal *glob);
static void add_rte_to_flat_rtable(PlannerGlobal *glob, RangeTblEntry *rte);
+static void add_target_partition_oids_recurse(Oid relid, int lockmode,
+ PlannerGlobal *glob);
static Plan *set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset);
static Plan *set_indexonlyscan_references(PlannerInfo *root,
IndexOnlyScan *plan,
@@ -261,6 +267,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If inserting into a partitioned table, add its partitions too to
+ * glob->relationOids to register them as plan dependencies.
+ */
+ if (root->parse->commandType == CMD_INSERT && glob->parallelModeNeeded)
+ {
+ RangeTblEntry *result_rte = rt_fetch(root->parse->resultRelation,
+ root->parse->rtable);
+
+ if (result_rte->relkind == RELKIND_PARTITIONED_TABLE)
+ add_target_partition_oids_recurse(result_rte->relid,
+ result_rte->rellockmode,
+ glob);
+ }
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
@@ -495,6 +516,39 @@ add_rte_to_flat_rtable(PlannerGlobal *glob, RangeTblEntry *rte)
glob->relationOids = lappend_oid(glob->relationOids, newrte->relid);
}
+/*
+ * Recursively adds a table's partitions' OIDs to glob->relationOids.
+ */
+static void
+add_target_partition_oids_recurse(Oid relid, int lockmode,
+ PlannerGlobal *glob)
+{
+ Relation rel;
+ PartitionDesc pdesc;
+ int i;
+
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ rel = table_open(relid, lockmode);
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ glob->relationOids = lappend_oid(glob->relationOids, pdesc->oids[i]);
+
+ /*
+ * If the partition may have its own partitions, recurse to add them
+ * too.
+ */
+ if (!pdesc->is_leaf[i])
+ add_target_partition_oids_recurse(pdesc->oids[i], lockmode, glob);
+ }
+
+ table_close(rel, NoLock);
+}
+
/*
* set_plan_refs: recurse through the Plan nodes of a single subquery level
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index de16a28639..2d681a6940 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1120,10 +1120,6 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
context);
table_close(part_rel, AccessShareLock);
- /* Register the partition as a plan dependency. */
- glob->relationOids =
- lappend_oid(glob->relationOids, pdesc->oids[i]);
-
if (max_hazard_found)
{
return true;
On Thu, Feb 18, 2021 at 12:34 AM Amit Langote <amitlangote09@gmail.com> wrote:
Your revised version seems OK, though I do have a concern:
Is the use of "table_close(rel, NoLock)'' intentional? That will keep
the lock (lockmode) until end-of-transaction.I think we always keep any locks on relations that are involved in a
plan until end-of-transaction. What if a partition is changed in an
unsafe manner between being considered safe for parallel insertion and
actually performing the parallel insert?BTW, I just noticed that exctract_query_dependencies() runs on a
rewritten, but not-yet-planned query tree, that is, I didn't know that
extract_query_dependencies() only populates the CachedPlanSource's
relationOids and not CachedPlan's. The former is only for tracking
the dependencies of an unplanned Query, so partitions should never be
added to it. Instead, they should be added to
PlannedStmt.relationOids (note PlannedStmt belongs to CachedPlan),
which is kind of what your earlier patch did. Needless to say,
PlanCacheRelCallback checks both CachedPlanSource.relationOids and
PlannedStmt.relationOids, so if it receives a message about a
partition, its OID is matched from the latter.All that is to say that we should move our code to add partition OIDs
as plan dependencies to somewhere in set_plan_references(), which
otherwise populates PlannedStmt.relationOids. I updated the patch to
do that.
OK, understood. Thanks for the detailed explanation.
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.
I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Feb 18, 2021 at 10:03 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Feb 18, 2021 at 12:34 AM Amit Langote <amitlangote09@gmail.com> wrote:
All that is to say that we should move our code to add partition OIDs
as plan dependencies to somewhere in set_plan_references(), which
otherwise populates PlannedStmt.relationOids. I updated the patch to
do that.OK, understood. Thanks for the detailed explanation.
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.
Ah, okay, I didn't retest my case after making that change.
Looking at this again, I am a bit concerned about going over the whole
partition tree *twice* when making a parallel plan for insert into
partitioned tables. Maybe we should do what you did in your first
attempt a slightly differently -- add partition OIDs during the
max_parallel_hazard() initiated scan of the partition tree as you did.
Instead of adding them directly to PlannerGlobal.relationOids, add to,
say, PlannerInfo.targetPartitionOids and have set_plan_references() do
list_concat(glob->relationOids, list_copy(root->targetPartitionOids)
in the same place as setrefs-v3 does
add_target_partition_oids_recurse(). Thoughts?
--
Amit Langote
EDB: http://www.enterprisedb.com
On Thu, Feb 18, 2021 at 4:35 PM Amit Langote <amitlangote09@gmail.com> wrote:
Looking at this again, I am a bit concerned about going over the whole
partition tree *twice* when making a parallel plan for insert into
partitioned tables. Maybe we should do what you did in your first
attempt a slightly differently -- add partition OIDs during the
max_parallel_hazard() initiated scan of the partition tree as you did.
Instead of adding them directly to PlannerGlobal.relationOids, add to,
say, PlannerInfo.targetPartitionOids and have set_plan_references() do
list_concat(glob->relationOids, list_copy(root->targetPartitionOids)
in the same place as setrefs-v3 does
add_target_partition_oids_recurse(). Thoughts?
Agreed, that might be a better approach, and that way we're also only
recording the partition OIDs that the parallel-safety checks are
relying on.
I'll give it a go and see if I can detect any issues with this method.
Regards,
Greg Nancarrow
Fujitsu Australia
Posting a new version of the patches, with the following updates:
- Moved the update of glob->relationOIDs (i.e. addition of partition
OIDs that plan depends on, resulting from parallel-safety checks) from
within max_parallel_hazard() to set_plan_references().
- Added an extra test for partition plan-cache invalidation.
- Simplified query_has_modifying_cte() temporary bug-fix.
- Added a comment explaining why parallel-safety of partition column
defaults is not checked.
- Minor simplification: hasSubQuery return to is_parallel_possible_for_modify().
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v18-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v18-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 717246cc3a9ddf25ab6c069a03e0954617043ab6 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 18 Feb 2021 12:35:25 +1100
Subject: [PATCH v18 2/4] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1240 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 612 ++++++++
5 files changed, 1857 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..e00b691315
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1240 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+drop trigger ai_rp2 on rp2;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 12bb67e491..37dc86359b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 59b416fd80..4504af73cb 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..bdda4dc6d2
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,612 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+drop trigger ai_rp2 on rp2;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
--
2.27.0
v18-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v18-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 1b3c7d245ecc7e9f65e89a70638fa02399c0b353 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 10 Feb 2021 14:50:40 +1100
Subject: [PATCH v18 3/4] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 28 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 511 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9926e2bd54..1da8d8379c 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 2456a2bf10..b241ea5ea9 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..f7c13bed7a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -212,6 +212,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index cf3604a7ea..b42e05c8b2 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -339,10 +342,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1811,7 +1815,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1859,6 +1976,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2299,96 +2417,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2407,6 +2462,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7575,7 +7637,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index f7d91c67bb..48467d6297 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -257,6 +257,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -319,7 +320,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1067,6 +1070,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 943931f65d..86e811f974 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..9f15fcb240 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -69,6 +69,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..2a41a00f29 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -575,15 +576,15 @@ typedef struct ViewOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 9a6720c2ed0906f3f2f12dd30390945933afa972 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 19 Feb 2021 11:20:56 +1100
Subject: [PATCH v18 1/4] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe trigger, index expression, column default
expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
This patch includes a TEMPORARY fix for a bug in the query rewriter.
Discussion: https://www.postgresql-archive.org/Bug-in-query-rewriter-hasModifyingCTE-not-getting-set-td6176917.html
It was found that for re-written queries with a modifying CTE, the
hasModifyingCTE flag was not getting set to true - resulting in a test failure
in the "with" tests when force_parallel_mode=regress was in effect, as it
allowed a CTE with a DELETE statement to be executed in parallel-mode.
This TEMPORARY fix (made in the planner) will need to be removed once the bug in
query rewriter is fixed. Note that this fix is currently just done for INSERT,
but the problem actually exists for SELECT too.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 93 ++++-
src/backend/optimizer/plan/setrefs.c | 17 +
src/backend/optimizer/util/clauses.c | 490 ++++++++++++++++++++++++++-
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/optimizer/clauses.h | 4 +-
8 files changed, 630 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 17fbc41bbb..2456a2bf10 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index adf68d8790..cf3604a7ea 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -246,7 +246,7 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *targetList,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
+static bool query_has_modifying_cte(Query *parse);
/*****************************************************************************
*
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -313,19 +314,35 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !parse->hasModifyingCTE)
+ {
+ /*
+ * FIXME
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this temporary fix must be removed.
+ *
+ * Note that here we've made a fix for this problem only for a
+ * supported-in-parallel-mode table-modification statement (i.e.
+ * INSERT), but this bug exists for SELECT too.
+ */
+ parse->hasModifyingCTE = query_has_modifying_cte(parse);
+ }
+
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +351,15 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -7768,3 +7787,53 @@ group_by_has_partkey(RelOptInfo *input_rel,
return true;
}
+
+/*
+ * Determine if the specified query has a modifying-CTE.
+ *
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this function should be removed.
+ *
+ */
+static bool
+query_has_modifying_cte(Query *parsetree)
+{
+ int rt_index;
+ bool hasModifyingCTE = false;
+
+ /* Recursively check subqueries */
+ rt_index = 0;
+ while (rt_index < list_length(parsetree->rtable))
+ {
+ RangeTblEntry *rte;
+ ++rt_index;
+ rte = rt_fetch(rt_index, parsetree->rtable);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasModifyingCTE = query_has_modifying_cte(rte->subquery);
+ if (hasModifyingCTE)
+ break;
+ }
+ }
+
+ if (!hasModifyingCTE)
+ {
+ ListCell *lc;
+
+ /* Check for INSERT/UPDATE/DELETE CTEs */
+ foreach(lc, parsetree->cteList)
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ hasModifyingCTE = true;
+ break;
+ }
+ }
+ }
+
+ return hasModifyingCTE;
+}
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..f7d91c67bb 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,10 @@
*/
#include "postgres.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +27,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -260,6 +265,18 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If modifying a partitioned table, add its parallel-safety-checked
+ * partitions too to glob->relationOids, to register them as plan
+ * dependencies.
+ */
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ if (glob->partitionOids != NIL)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+ }
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..af8b6d4437 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +170,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +566,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +609,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -757,6 +786,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +819,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +830,446 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_expr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions of a specified relation.
+ */
+static bool
+target_rel_index_expr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+
+ /* Record the partition as a plan dependency. */
+ glob->partitionOids =
+ lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions, check that they are parallel-mode
+ * safe.
+ */
+ if (target_rel_index_expr_max_parallel_hazard(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only parallel INSERT is currently supported,
+ * only command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan
+ * of the query tree in standard_planner().
+ * Note that even though column defaults may be specified separately for
+ * each partition in a partitioned table, a partition's default value is
+ * not applied when inserting a tuple through a partitioned table.
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_possible_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_possible_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ Assert(IsModifySupportedInParallelMode(parse->commandType));
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 0ec93e648c..6eabab4224 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..e2f0fe3aa6 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,7 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
+extern bool is_parallel_possible_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v18-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v18-0004-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From bb8c4aaf554f25fcce69533591cea8a1e37b98b3 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 18 Feb 2021 12:47:31 +1100
Subject: [PATCH v18 4/4] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++-
src/test/regress/expected/insert_parallel.out | 142 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 +--
3 files changed, 158 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index e00b691315..9e754a600e 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -77,14 +77,14 @@ create table para_insert_f1 (
);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -120,7 +120,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -167,15 +167,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -341,9 +341,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -391,9 +391,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -426,9 +426,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -462,9 +462,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -500,9 +500,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -540,9 +540,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -614,21 +614,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -657,7 +657,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -710,14 +710,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -766,7 +766,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -795,14 +795,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ create table parttable1_2 partition of parttable1 for values from (5000) to (100
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -962,7 +962,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -983,11 +983,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1028,7 +1028,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1085,13 +1085,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1114,9 +1114,9 @@ drop trigger ai_rp2 on rp2;
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1130,9 +1130,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index bdda4dc6d2..e7a2bdb521 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -97,7 +97,7 @@ create table para_insert_f1 (
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -109,7 +109,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -122,7 +122,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -266,14 +266,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -281,7 +281,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -307,7 +307,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -323,7 +323,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -331,7 +331,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -415,7 +415,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -465,7 +465,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -511,7 +511,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
From: Greg Nancarrow <gregn4422@gmail.com>
--------------------------------------------------
On Mon, Jan 25, 2021 at 10:23 AM tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com> wrote:
(8) + /* + * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in + * the relation, and this would result in creation of new CommandIds + * on insert/update/delete and this isn't supported in a parallel + * worker (but is safe in the parallel leader). + */ + trigtype = RI_FKey_trigger_type(trigger->tgfoid); + if (trigtype == RI_TRIGGER_FK) + { + if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)) + return true; + }Here, RI_TRIGGER_FK should instead be RI_TRIGGER_PK, because RI_TRIGGER_FK triggers do not generate command IDs. See RI_FKey_check() which is called in RI_TRIGGER_FK case. In there, ri_PerformCheck() is called with the detectNewRows argument set to false, which causes CommandCounterIncrement() to not be called.
Hmmm, I'm not sure that you have read and interpreted the patch code correctly.
The existence of a RI_TRIGGER_FK trigger indicates the table has a foreign key, and an insert into such a table will generate a new commandId (so we must avoid that, as we don't currently have the technology to support sharing of new command IDs across the participants in the parallel operation). This is what the code comment says, It does not say that such a trigger generates a new command ID.
See Amit's updated comment here: https://github.com/postgres/postgres/commit/0d32511eca5aec205cb6b609638ea67129ef6665
In addition, the 2nd patch has an explicit test case for this (testing insert into a table that has a FK).
--------------------------------------------------
First of all, I anticipate this parallel INSERT SELECT feature will typically shine, and expected to work, in the ETL or ELT into a data warehouse or an ODS for analytics. Bearing that in mind, let me list some issues or questions below. But the current state of the patch would be of course attractive in some workloads, so I don't think these are not necessarily blockers.
(1)
According to the classic book "The Data Warehouse Toolkit" and the website [1]Fact Table Surrogate Key | Kimball Dimensional Modeling Techniques https://www.kimballgroup.com/data-warehouse-business-intelligence-resources/kimball-techniques/dimensional-modeling-techniques/fact-surrogate-key/ by its author, the fact table (large transaction history) in the data warehouse has foreign keys referencing to the dimension tables (small or medium-sized master or reference data). So, parallel insert will be effective if it works when loading data into the fact table with foreign keys.
To answer the above question, I'm assuming:
CREATE TABLE some_dimension (key_col int PRIMARY KEY);
CREATE TABLE some_fact (some_key int REFERENCES some_dimension);
INSERT INTO some_fact SELECT ...;
My naive question is, "why should new command IDs be generated to check foreign key constraints in this INSERT case? The check just reads the parent (some_dimension table here)..."
Looking a bit deeper into the code, although ri_PerformCheck() itself tries to avoid generating command IDs, it calls _SPI_execute_snapshot() with the read_only argument always set to false. It in turn calls _SPI_execute_plan() -> CommandCounterIncrement() as follows:
[_SPI_execute_plan()]
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
*/
if (!read_only && !plan->no_snapshots)
{
CommandCounterIncrement();
UpdateActiveSnapshotCommandId();
}
Can't we pass true to read_only from ri_PerformCheck() in some cases?
(2)
Likewise, dimension tables have surrogate keys that are typically implemented as a sequence or an identity column. It is suggested that even fact tables sometimes (or often?) have surrogate keys. But the current patch does not parallelize the statement when the target table has a sequence or an identity column.
I was looking at the sequence code, and my naive (again) idea is that the parallel leader and workers allocates numbers from the sequence independently, and sets the largest number of them as the session's currval at the end of parallel operation. We have to note in the documentation that gaps in the sequence numbers will arise and not used in parallel DML.
(3)
As Hou-san demonstrated, the current patch causes the resulting table and index to become larger when inserted in parallel than in inserted serially. This could be a problem for analytics use cases where the table is just inserted and read only afterwards. We could advise the user to run REINDEX CONCURRENTLY after loading data, but what about tables?
BTW, I don't know if Oracle and SQL Server have similar issues. They may have some reason about this why they take an exclusive lock on the target table.
(4)
When the target table is partitioned, is the INSERT parallelized among its partitions? Some plan like:
Parallel Append on parent_table
-> Insert on partiton1
-> Insert on partiton2
[1]: Fact Table Surrogate Key | Kimball Dimensional Modeling Techniques https://www.kimballgroup.com/data-warehouse-business-intelligence-resources/kimball-techniques/dimensional-modeling-techniques/fact-surrogate-key/
Fact Table Surrogate Key | Kimball Dimensional Modeling Techniques
https://www.kimballgroup.com/data-warehouse-business-intelligence-resources/kimball-techniques/dimensional-modeling-techniques/fact-surrogate-key/
Regards
Takayuki Tsunakawa
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).
Also, if we agree that we don't have any cheap way to determine
parallel-safety of partitioned relations then shall we consider the
patch being discussed [1]/messages/by-id/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ@mail.gmail.com to be combined here?
Amit L, do you agree with that line of thought, or you have any other ideas?
I feel we should focus on getting the first patch of Greg
(v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT, along with
a test case patch) and Hou-San's patch discussed at [1]/messages/by-id/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ@mail.gmail.com ready. Then we
can focus on the
v18-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO. Because
even if we get the first patch that is good enough for some users.
What do you think?
[1]: /messages/by-id/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ@mail.gmail.com
--
With Regards,
Amit Kapila.
On Fri, Feb 19, 2021 at 10:13 AM tsunakawa.takay@fujitsu.com
<tsunakawa.takay@fujitsu.com> wrote:
From: Greg Nancarrow <gregn4422@gmail.com>
--------------------------------------------------
On Mon, Jan 25, 2021 at 10:23 AM tsunakawa.takay@fujitsu.com <tsunakawa.takay@fujitsu.com> wrote:(8) + /* + * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in + * the relation, and this would result in creation of new CommandIds + * on insert/update/delete and this isn't supported in a parallel + * worker (but is safe in the parallel leader). + */ + trigtype = RI_FKey_trigger_type(trigger->tgfoid); + if (trigtype == RI_TRIGGER_FK) + { + if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context)) + return true; + }Here, RI_TRIGGER_FK should instead be RI_TRIGGER_PK, because RI_TRIGGER_FK triggers do not generate command IDs. See RI_FKey_check() which is called in RI_TRIGGER_FK case. In there, ri_PerformCheck() is called with the detectNewRows argument set to false, which causes CommandCounterIncrement() to not be called.
Hmmm, I'm not sure that you have read and interpreted the patch code correctly.
The existence of a RI_TRIGGER_FK trigger indicates the table has a foreign key, and an insert into such a table will generate a new commandId (so we must avoid that, as we don't currently have the technology to support sharing of new command IDs across the participants in the parallel operation). This is what the code comment says, It does not say that such a trigger generates a new command ID.See Amit's updated comment here: https://github.com/postgres/postgres/commit/0d32511eca5aec205cb6b609638ea67129ef6665
In addition, the 2nd patch has an explicit test case for this (testing insert into a table that has a FK).
--------------------------------------------------First of all, I anticipate this parallel INSERT SELECT feature will typically shine, and expected to work, in the ETL or ELT into a data warehouse or an ODS for analytics. Bearing that in mind, let me list some issues or questions below. But the current state of the patch would be of course attractive in some workloads, so I don't think these are not necessarily blockers.
(1)
According to the classic book "The Data Warehouse Toolkit" and the website [1] by its author, the fact table (large transaction history) in the data warehouse has foreign keys referencing to the dimension tables (small or medium-sized master or reference data). So, parallel insert will be effective if it works when loading data into the fact table with foreign keys.To answer the above question, I'm assuming:
CREATE TABLE some_dimension (key_col int PRIMARY KEY);
CREATE TABLE some_fact (some_key int REFERENCES some_dimension);
INSERT INTO some_fact SELECT ...;My naive question is, "why should new command IDs be generated to check foreign key constraints in this INSERT case? The check just reads the parent (some_dimension table here)..."
It is quite possible what you are saying is correct but I feel that is
not this patch's fault. So, won't it better to discuss this in a
separate thread?
(2)
Likewise, dimension tables have surrogate keys that are typically implemented as a sequence or an identity column. It is suggested that even fact tables sometimes (or often?) have surrogate keys. But the current patch does not parallelize the statement when the target table has a sequence or an identity column.I was looking at the sequence code, and my naive (again) idea is that the parallel leader and workers allocates numbers from the sequence independently, and sets the largest number of them as the session's currval at the end of parallel operation. We have to note in the documentation that gaps in the sequence numbers will arise and not used in parallel DML.
Good use case but again, I think this can be done as a separate patch.
(3)
As Hou-san demonstrated, the current patch causes the resulting table and index to become larger when inserted in parallel than in inserted serially. This could be a problem for analytics use cases where the table is just inserted and read only afterwards. We could advise the user to run REINDEX CONCURRENTLY after loading data, but what about tables?
I think here you are talking about the third patch (Parallel Inserts).
I guess if one has run inserts parallelly from psql then also similar
behavior would have been observed. For tables, it might lead to better
results once we have the patch discussed at [1]/messages/by-id/20200508072545.GA9701@telsasoft.com. Actually, this needs
more investigation.
[1]: /messages/by-id/20200508072545.GA9701@telsasoft.com
--
With Regards,
Amit Kapila.
From: Amit Kapila <amit.kapila16@gmail.com>
It is quite possible what you are saying is correct but I feel that is
not this patch's fault. So, won't it better to discuss this in a
separate thread?Good use case but again, I think this can be done as a separate patch.
Agreed.
I think even the current patch offers great benefits and can be committed in PG 14, even if all my four feedback comments are left unaddressed. I just touched on them for completeness in terms of typically expected use cases. They will probably be able to be implemented along the current design.
I think here you are talking about the third patch (Parallel Inserts).
I guess if one has run inserts parallelly from psql then also similar
behavior would have been observed. For tables, it might lead to better
results once we have the patch discussed at [1]. Actually, this needs
more investigation.
That looks interesting and worth a try.
Regards
Takayuki Tsunakawa
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).
I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan. This would seem to go against what you are
suggesting.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 19, 2021 at 7:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time.It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).
Are you saying that partitions shouldn't be added to the dependency
list if a parallel plan was not chosen for insert into a partitioned
table for whatever reason (parallel unsafe expressions or beaten by
other paths in terms of cost)? If so, I am inclined to agree with
that.
I may be wrong but it doesn't seem to me that the possibility of
constructing a better plan due to a given change is enough reason for
plancache.c to invalidate plans that depend on that change. AIUI,
plancache.c only considers a change interesting if it would *break* a
Query or a plan.
So in this case, a non-parallel plan may be slower, but it isn't
exactly rendered *wrong* by changes that make a parallel plan
possible.
Also, if we agree that we don't have any cheap way to determine
parallel-safety of partitioned relations then shall we consider the
patch being discussed [1] to be combined here?
Yes, I think it does make sense to consider the GUC patch with the
patches on this thread.
I feel we should focus on getting the first patch of Greg
(v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT, along with
a test case patch) and Hou-San's patch discussed at [1] ready. Then we
can focus on the
v18-0003-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO. Because
even if we get the first patch that is good enough for some users.
+1.
--
Amit Langote
EDB: http://www.enterprisedb.com
Posting a new version of the patches, with the following updates:
- Moved the update of glob->relationOIDs (i.e. addition of partition OIDs that
plan depends on, resulting from parallel-safety checks) from within
max_parallel_hazard() to set_plan_references().
- Added an extra test for partition plan-cache invalidation.
- Simplified query_has_modifying_cte() temporary bug-fix.
- Added a comment explaining why parallel-safety of partition column defaults
is not checked.
- Minor simplification: hasSubQuery return to
is_parallel_possible_for_modify().
Hi
(I may be wrong here)
I noticed that the patch does not have check for partial index(index predicate).
It seems parallel index build will check it like the following:
----------
/*
* Determine if it's safe to proceed.
*
* Currently, parallel workers can't access the leader's temporary tables.
* Furthermore, any index predicate or index expressions must be parallel
* safe.
*/
if (heap->rd_rel->relpersistence == RELPERSISTENCE_TEMP ||
!is_parallel_safe(root, (Node *) RelationGetIndexExpressions(index)) ||
!is_parallel_safe(root, (Node *) RelationGetIndexPredicate(index)))
----------
Should we do parallel safety check for it ?
Best regards,
houzj
On Mon, Feb 22, 2021 at 6:25 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
Hi
(I may be wrong here)
I noticed that the patch does not have check for partial index(index predicate).
It seems parallel index build will check it like the following:
----------
/*
* Determine if it's safe to proceed.
*
* Currently, parallel workers can't access the leader's temporary tables.
* Furthermore, any index predicate or index expressions must be parallel
* safe.
*/
if (heap->rd_rel->relpersistence == RELPERSISTENCE_TEMP ||
!is_parallel_safe(root, (Node *) RelationGetIndexExpressions(index)) ||
!is_parallel_safe(root, (Node *) RelationGetIndexPredicate(index)))
----------Should we do parallel safety check for it ?
Thanks, it looks like you're right, it is missing (and there's no test for it).
I can add a fix to the index-checking code, something like:
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *)ii_Predicate, context))
+ {
+ found_max_hazard = true;
+ }
+ }
+ }
Also will need a bit of renaming of that function.
I'll include this in the next patch update.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Feb 22, 2021 at 8:46 AM Amit Langote <amitlangote09@gmail.com> wrote:
On Fri, Feb 19, 2021 at 7:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time.It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).Are you saying that partitions shouldn't be added to the dependency
list if a parallel plan was not chosen for insert into a partitioned
table for whatever reason (parallel unsafe expressions or beaten by
other paths in terms of cost)?
Right.
If so, I am inclined to agree with
that.I may be wrong but it doesn't seem to me that the possibility of
constructing a better plan due to a given change is enough reason for
plancache.c to invalidate plans that depend on that change. AIUI,
plancache.c only considers a change interesting if it would *break* a
Query or a plan.
that makes sense to me.
So in this case, a non-parallel plan may be slower, but it isn't
exactly rendered *wrong* by changes that make a parallel plan
possible.
Right.
Also, if we agree that we don't have any cheap way to determine
parallel-safety of partitioned relations then shall we consider the
patch being discussed [1] to be combined here?Yes, I think it does make sense to consider the GUC patch with the
patches on this thread.
Cool, Greg/Hou, can we consider this along with the next patch?
--
With Regards,
Amit Kapila.
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.
Does such a plan have partitions access in it? Can you share the plan?
--
With Regards,
Amit Kapila.
On Tue, Feb 23, 2021 at 12:33 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.Does such a plan have partitions access in it? Can you share the plan?
Er, no (it's just a regular table), but that was exactly my point:
aren't you suggesting functionality for partitions that doesn't seem
to work the same way for non-partitions?
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Feb 23, 2021 at 6:37 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 12:33 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.Does such a plan have partitions access in it? Can you share the plan?
Er, no (it's just a regular table), but that was exactly my point:
aren't you suggesting functionality for partitions that doesn't seem
to work the same way for non-partitions?
I don't think so. The non-parallel plan for Insert doesn't directly
depend on partitions so we don't need to invalidate those.
--
With Regards,
Amit Kapila.
On Tue, Feb 23, 2021 at 2:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Feb 23, 2021 at 6:37 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 12:33 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.Does such a plan have partitions access in it? Can you share the plan?
Er, no (it's just a regular table), but that was exactly my point:
aren't you suggesting functionality for partitions that doesn't seem
to work the same way for non-partitions?I don't think so. The non-parallel plan for Insert doesn't directly
depend on partitions so we don't need to invalidate those.
But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.
I know you're suggesting to reduce plan-cache invalidation by only
recording a dependency in the parallel-plan case, but I've yet to see
that in the existing code, and in fact it seems to be inconsistent
with the behaviour I've tested so far (one example given prior, but
will look for more).
Regards,
Greg Nancarrow
Fujitsu Australia
On Tue, Feb 23, 2021 at 4:47 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 2:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Feb 23, 2021 at 6:37 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 12:33 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.Does such a plan have partitions access in it? Can you share the plan?
Er, no (it's just a regular table), but that was exactly my point:
aren't you suggesting functionality for partitions that doesn't seem
to work the same way for non-partitions?I don't think so. The non-parallel plan for Insert doesn't directly
depend on partitions so we don't need to invalidate those.But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.
Hmm, I think that is not what we can consider as a dependency.
I know you're suggesting to reduce plan-cache invalidation by only
recording a dependency in the parallel-plan case, but I've yet to see
that in the existing code, and in fact it seems to be inconsistent
with the behaviour I've tested so far (one example given prior, but
will look for more).
I don't see your example matches what you are saying as in it the
regular table exists in the plan whereas for the case we are
discussing partitions doesn't exist in the plan. Amit L. seems to have
given a correct explanation [1]/messages/by-id/CA+HiwqFKJfzgBbkg0i0Fz_FGsCiXW-Fw0tBjdsaUbNbpyv0JhA@mail.gmail.com of why we don't need to invalidate for
non-parallel plans which match my understanding.
[1]: /messages/by-id/CA+HiwqFKJfzgBbkg0i0Fz_FGsCiXW-Fw0tBjdsaUbNbpyv0JhA@mail.gmail.com
--
With Regards,
Amit Kapila.
On Tue, Feb 23, 2021 at 10:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Feb 23, 2021 at 4:47 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 2:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Feb 23, 2021 at 6:37 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 12:33 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 22, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 19, 2021 at 9:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Feb 18, 2021 at 11:05 AM Amit Langote <amitlangote09@gmail.com> wrote:
It also occurred to me that we can avoid pointless adding of
partitions if the final plan won't use parallelism. For that, the
patch adds checking glob->parallelModeNeeded, which seems to do the
trick though I don't know if that's the correct way of doing that.I'm not sure if's pointless adding partitions even in the case of NOT
using parallelism, because we may be relying on the result of
parallel-safety checks on partitions in both cases.
For example, insert_parallel.sql currently includes a test (that you
originally provided in a previous post) that checks a non-parallel
plan is generated after a parallel-unsafe trigger is created on a
partition involved in the INSERT.
If I further add to that test by then dropping that trigger and then
again using EXPLAIN to see what plan is generated, then I'd expect a
parallel-plan to be generated, but with the setrefs-v3.patch it still
generates a non-parallel plan. So I think the "&&
glob->parallelModeNeeded" part of test needs to be removed.Ah, okay, I didn't retest my case after making that change.
Greg has point here but I feel something on previous lines (having a
test of glob->parallelModeNeeded) is better. We only want to
invalidate the plan if the prepared plan is unsafe to execute next
time. It is quite possible that there are unsafe triggers on different
partitions and only one of them is dropped, so next time planning will
again yield to the same non-parallel plan. If we agree with that I
think it is better to add this dependency in set_plan_refs (along with
Gather node handling).I think we should try to be consistent with current plan-cache
functionality, rather than possibly inventing new rules for
partitions.
I'm finding that with current Postgres functionality (master branch),
if there are two parallel-unsafe triggers defined on a normal table
and one is dropped, it results in replanning and it yields the same
(non-parallel) plan.Does such a plan have partitions access in it? Can you share the plan?
Er, no (it's just a regular table), but that was exactly my point:
aren't you suggesting functionality for partitions that doesn't seem
to work the same way for non-partitions?I don't think so. The non-parallel plan for Insert doesn't directly
depend on partitions so we don't need to invalidate those.But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.Hmm, I think that is not what we can consider as a dependency.
Then if it's not a dependency, then we shouldn't have to check the
attributes of the partitions for parallel-safety, to determine whether
we must use a non-parallel plan (or can use a parallel plan).
Except, of course, we do have to ...
I know you're suggesting to reduce plan-cache invalidation by only
recording a dependency in the parallel-plan case, but I've yet to see
that in the existing code, and in fact it seems to be inconsistent
with the behaviour I've tested so far (one example given prior, but
will look for more).I don't see your example matches what you are saying as in it the
regular table exists in the plan whereas for the case we are
discussing partitions doesn't exist in the plan. Amit L. seems to have
given a correct explanation [1] of why we don't need to invalidate for
non-parallel plans which match my understanding.[1] - /messages/by-id/CA+HiwqFKJfzgBbkg0i0Fz_FGsCiXW-Fw0tBjdsaUbNbpyv0JhA@mail.gmail.com
Amit L's explanation was:
I may be wrong but it doesn't seem to me that the possibility of
constructing a better plan due to a given change is enough reason for
plancache.c to invalidate plans that depend on that change. AIUI,
plancache.c only considers a change interesting if it would *break* a
Query or a plan.
So in this case, a non-parallel plan may be slower, but it isn't
exactly rendered *wrong* by changes that make a parallel plan
possible.
This explanation doesn't seem to match existing planner behavior
AFAICS, and we should try to be consistent with existing behavior
(rather than doing something different, for partitions specifically in
this case).
Using a concrete example, to avoid any confusion, consider the
following SQL (using unpatched Postgres, master branch):
-- encourage use of parallel plans, where possible
set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;
create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel unsafe immutable;
create table mytable(x int);
insert into mytable(x) values(generate_series(1,10));
prepare q as select * from mytable, myfunc();
explain execute q;
-- change myfunc to be parallel-safe, to see the effect
-- on the planner for the same select
create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel safe immutable;
explain execute q;
Here a function referenced in the SELECT statement is changed from
parallel-unsafe to parallel-safe, to see the effect on plancache.
According to your referenced explanation, that shouldn't be considered
an "interesting" change by plancache.c, as it wouldn't "break" the
previously planned and cached (non-parallel) query - the existing
non-parallel plan could and should be used, as it still would execute
OK, even if slower. Right?
BUT - for the above example, it DOES cause the query to be replanned,
and it then uses a parallel plan, instead of keeping the original
non-parallel plan.
This does not match the explanation about how plancache works.
Output is below:
test=# \i test.sql
-- encourage use of parallel plans, where possible
set parallel_setup_cost=0;
SET
set parallel_tuple_cost=0;
SET
set min_parallel_table_scan_size=0;
SET
set max_parallel_workers_per_gather=4;
SET
create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel unsafe immutable;
CREATE FUNCTION
create table mytable(x int);
CREATE TABLE
insert into mytable(x) values(generate_series(1,10));
INSERT 0 10
prepare q as select * from mytable, myfunc();
PREPARE
explain execute q;
QUERY PLAN
-----------------------------------------------------------
Seq Scan on mytable (cost=0.00..35.50 rows=2550 width=5)
(1 row)
-- change myfunc to be parallel-safe, to see the effect
-- on the planner for the same select
create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel safe immutable;
CREATE FUNCTION
explain execute q;
QUERY PLAN
-------------------------------------------------------------------------
Gather (cost=0.00..18.23 rows=2550 width=5)
Workers Planned: 3
-> Parallel Seq Scan on mytable (cost=0.00..18.23 rows=823 width=5)
(3 rows)
So what I am saying is why should this behavior be different for our
partition case?
Doing something different for partitions would be inconsistent with
this behavior, would it not?
Going back to my original example, I pointed out that if there are two
parallel-unsafe triggers defined on a normal table and one is dropped,
it results in replanning and it yields the same (non-parallel) plan.
If the 2nd trigger is dropped, it also results in replanning (and with
patch applied, will yield a parallel plan).
And with the functionality you are suggesting (only add partition OIDs
as dependencies in the case of using a parallel plan), the behavior
for partitions (as opposed to a normal table) will be different. Drop
one or two of the parallel-unsafe triggers and it won't result in
replanning and will still use the original non-parallel plan.
So why should the behavior for partitions be different here?
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 24, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 10:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.Hmm, I think that is not what we can consider as a dependency.
Then if it's not a dependency, then we shouldn't have to check the
attributes of the partitions for parallel-safety, to determine whether
we must use a non-parallel plan (or can use a parallel plan).
Except, of course, we do have to ...
I don't think the plan-dependency and checking for parallel-safety are
directly related.
I know you're suggesting to reduce plan-cache invalidation by only
recording a dependency in the parallel-plan case, but I've yet to see
that in the existing code, and in fact it seems to be inconsistent
with the behaviour I've tested so far (one example given prior, but
will look for more).I don't see your example matches what you are saying as in it the
regular table exists in the plan whereas for the case we are
discussing partitions doesn't exist in the plan. Amit L. seems to have
given a correct explanation [1] of why we don't need to invalidate for
non-parallel plans which match my understanding.[1] - /messages/by-id/CA+HiwqFKJfzgBbkg0i0Fz_FGsCiXW-Fw0tBjdsaUbNbpyv0JhA@mail.gmail.com
Amit L's explanation was:
I may be wrong but it doesn't seem to me that the possibility of
constructing a better plan due to a given change is enough reason for
plancache.c to invalidate plans that depend on that change. AIUI,
plancache.c only considers a change interesting if it would *break* a
Query or a plan.So in this case, a non-parallel plan may be slower, but it isn't
exactly rendered *wrong* by changes that make a parallel plan
possible.This explanation doesn't seem to match existing planner behavior
AFAICS, and we should try to be consistent with existing behavior
(rather than doing something different, for partitions specifically in
this case).
I still think it matches. You have missed the important point in your
example and explanation which Amit L and I am trying to explain to
you. See below.
Using a concrete example, to avoid any confusion, consider the
following SQL (using unpatched Postgres, master branch):-- encourage use of parallel plans, where possible
set parallel_setup_cost=0;
set parallel_tuple_cost=0;
set min_parallel_table_scan_size=0;
set max_parallel_workers_per_gather=4;create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel unsafe immutable;create table mytable(x int);
insert into mytable(x) values(generate_series(1,10));prepare q as select * from mytable, myfunc();
explain execute q;-- change myfunc to be parallel-safe, to see the effect
-- on the planner for the same select
create or replace function myfunc() returns boolean as $$
begin
return true;
end;
$$ language plpgsql parallel safe immutable;explain execute q;
Here a function referenced in the SELECT statement is changed from
parallel-unsafe to parallel-safe, to see the effect on plancache.
According to your referenced explanation, that shouldn't be considered
an "interesting" change by plancache.c, as it wouldn't "break" the
previously planned and cached (non-parallel) query - the existing
non-parallel plan could and should be used, as it still would execute
OK, even if slower. Right?
No that is not what I said or maybe you have misunderstood. In your
example, if you use the verbose option, then you will see the output
as below.
postgres=# explain (verbose) execute q;
QUERY PLAN
------------------------------------------------------------------
Seq Scan on public.mytable (cost=0.00..35.50 rows=2550 width=5)
Output: mytable.x, true
(2 rows)
Here, you can see that Plan depends on function (it's return value),
so it needs to invalidated when the function changes. To, see it in a
bit different way, if you change your function as below then, you can
clearly see FunctionScan in the plan:
create or replace function myfunc(c1 int) returns int as $$
begin
return c1 * 10;
end;
$$ language plpgsql parallel unsafe;
postgres=# prepare q as select * from mytable, myfunc(2);
PREPARE
postgres=# explain select * from mytable, myfunc(2);
QUERY PLAN
-----------------------------------------------------------------
Nested Loop (cost=0.25..61.26 rows=2550 width=8)
-> Function Scan on myfunc (cost=0.25..0.26 rows=1 width=4)
-> Seq Scan on mytable (cost=0.00..35.50 rows=2550 width=4)
(3 rows)
BUT - for the above example, it DOES cause the query to be replanned,
and it then uses a parallel plan, instead of keeping the original
non-parallel plan.
This does not match the explanation about how plancache works.
As explained above and as far as I can understand it matches Amit L's
explanation because the plan depends on the function output.
So what I am saying is why should this behavior be different for our
partition case?
Doing something different for partitions would be inconsistent with
this behavior, would it not?
No, please show an example of the Insert case where the plan has
reference to partitions. If that happens then we should add those as a
dependency but AFAICT that is not the case here.
--
With Regards,
Amit Kapila.
On Wed, Feb 24, 2021 at 3:12 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 24, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 10:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.Hmm, I think that is not what we can consider as a dependency.
Then if it's not a dependency, then we shouldn't have to check the
attributes of the partitions for parallel-safety, to determine whether
we must use a non-parallel plan (or can use a parallel plan).
Except, of course, we do have to ...I don't think the plan-dependency and checking for parallel-safety are
directly related.
That is certainly not my understanding. Why do you think that they are
not directly related?
This whole issue came about because Amit L pointed out that there is a
need to add partition OIDs as plan-dependencies BECAUSE the checking
for parallel-safety and plan-dependency are related - since now, for
Parallel INSERT, we're executing extra parallel-safety checks that
check partition properties, so the resultant plan is dependent on the
partitions and their properties.
Amit L originally explained this as follows:
"I think that the concerns raised by Tsunakawa-san in:
/messages/by-id/TYAPR01MB2990CCB6E24B10D35D28B949FEA30@TYAPR01MB2990.jpnprd01.prod.outlook.com
regarding how this interacts with plancache.c deserve a look.
Specifically, a plan that uses parallel insert may fail to be
invalidated when partitions are altered directly (that is without
altering their root parent). That would be because we are not adding
partition OIDs to PlannerGlobal.invalItems despite making a plan
that's based on checking their properties."
Tsunakawa-san: "Is the cached query plan invalidated when some
alteration is done to change the parallel safety, such as adding a
trigger with a parallel-unsafe trigger action?"
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 24, 2021 at 2:14 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 24, 2021 at 3:12 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 24, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 10:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.Hmm, I think that is not what we can consider as a dependency.
Then if it's not a dependency, then we shouldn't have to check the
attributes of the partitions for parallel-safety, to determine whether
we must use a non-parallel plan (or can use a parallel plan).
Except, of course, we do have to ...I don't think the plan-dependency and checking for parallel-safety are
directly related.That is certainly not my understanding. Why do you think that they are
not directly related?
This whole issue came about because Amit L pointed out that there is a
need to add partition OIDs as plan-dependencies BECAUSE the checking
for parallel-safety and plan-dependency are related - since now, for
Parallel INSERT, we're executing extra parallel-safety checks that
check partition properties, so the resultant plan is dependent on the
partitions and their properties.
He has pointed out an issue when the plan is parallel and you can see
in that example that it fails if we didn't invalidate it. For
non-parallel plans, there won't be any such issue.
--
With Regards,
Amit Kapila.
On Fri, Feb 19, 2021 at 6:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting a new version of the patches, with the following updates:
I am not happy with the below code changes, I think we need a better
way to deal with this.
@@ -313,19 +314,35 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !parse->hasModifyingCTE)
+ {
+ /*
+ * FIXME
+ * There is a known bug in the query rewriter: re-written queries with
+ * a modifying CTE may not have the "hasModifyingCTE" flag set. When
+ * that bug is fixed, this temporary fix must be removed.
+ *
+ * Note that here we've made a fix for this problem only for a
+ * supported-in-parallel-mode table-modification statement (i.e.
+ * INSERT), but this bug exists for SELECT too.
+ */
+ parse->hasModifyingCTE = query_has_modifying_cte(parse);
+ }
+
I understand that this is an existing bug but I am not happy with this
workaround. I feel it is better to check for modifyingCTE in
max_parallel_hazard_walker. See attached, this is atop
v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.
--
With Regards,
Amit Kapila.
Attachments:
fix_modifyingcte_parallel_safety_v1.patchapplication/octet-stream; name=fix_modifyingcte_parallel_safety_v1.patchDownload
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b4b85fd681..e60b484f47 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -246,7 +246,6 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *targetList,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-static bool query_has_modifying_cte(Query *parse);
/*****************************************************************************
*
@@ -314,22 +313,6 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;
- if (IsModifySupportedInParallelMode(parse->commandType) &&
- !parse->hasModifyingCTE)
- {
- /*
- * FIXME
- * There is a known bug in the query rewriter: re-written queries with
- * a modifying CTE may not have the "hasModifyingCTE" flag set. When
- * that bug is fixed, this temporary fix must be removed.
- *
- * Note that here we've made a fix for this problem only for a
- * supported-in-parallel-mode table-modification statement (i.e.
- * INSERT), but this bug exists for SELECT too.
- */
- parse->hasModifyingCTE = query_has_modifying_cte(parse);
- }
-
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
@@ -7788,53 +7771,3 @@ group_by_has_partkey(RelOptInfo *input_rel,
return true;
}
-
-/*
- * Determine if the specified query has a modifying-CTE.
- *
- * There is a known bug in the query rewriter: re-written queries with
- * a modifying CTE may not have the "hasModifyingCTE" flag set. When
- * that bug is fixed, this function should be removed.
- *
- */
-static bool
-query_has_modifying_cte(Query *parsetree)
-{
- int rt_index;
- bool hasModifyingCTE = false;
-
- /* Recursively check subqueries */
- rt_index = 0;
- while (rt_index < list_length(parsetree->rtable))
- {
- RangeTblEntry *rte;
- ++rt_index;
- rte = rt_fetch(rt_index, parsetree->rtable);
- if (rte->rtekind == RTE_SUBQUERY)
- {
- hasModifyingCTE = query_has_modifying_cte(rte->subquery);
- if (hasModifyingCTE)
- break;
- }
- }
-
- if (!hasModifyingCTE)
- {
- ListCell *lc;
-
- /* Check for INSERT/UPDATE/DELETE CTEs */
- foreach(lc, parsetree->cteList)
- {
- CommonTableExpr *cte = (CommonTableExpr *) lfirst(lc);
- Query *ctequery = castNode(Query, cte->ctequery);
-
- if (ctequery->commandType != CMD_SELECT)
- {
- hasModifyingCTE = true;
- break;
- }
- }
- }
-
- return hasModifyingCTE;
-}
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index af8b6d4437..4a918068e8 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -716,6 +716,25 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, the Query tree has hasModifyingCTE flag that indicates
+ * the statement is parallel-unsafe but in some cases where query with
+ * modifying CTE is re-written that flag is not set.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -796,6 +815,8 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
if (rte == context->target_rte)
return target_rel_max_parallel_hazard(context);
+ else if (rte->rtekind == RTE_SUBQUERY)
+ return max_parallel_hazard_walker((Node *) rte->subquery, context);
return false;
}
On Wed, Feb 24, 2021 at 8:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 19, 2021 at 6:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting a new version of the patches, with the following updates:
I am not happy with the below code changes, I think we need a better
way to deal with this.@@ -313,19 +314,35 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;+ if (IsModifySupportedInParallelMode(parse->commandType) && + !parse->hasModifyingCTE) + { + /* + * FIXME + * There is a known bug in the query rewriter: re-written queries with + * a modifying CTE may not have the "hasModifyingCTE" flag set. When + * that bug is fixed, this temporary fix must be removed. + * + * Note that here we've made a fix for this problem only for a + * supported-in-parallel-mode table-modification statement (i.e. + * INSERT), but this bug exists for SELECT too. + */ + parse->hasModifyingCTE = query_has_modifying_cte(parse); + } +I understand that this is an existing bug but I am not happy with this
workaround. I feel it is better to check for modifyingCTE in
max_parallel_hazard_walker. See attached, this is atop
v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.
Thanks, I'll try it.
I did, however, notice a few concerns with your suggested alternative fix:
- It is not restricted to INSERT (as current fix is).
- It does not set parse->hasModifyingCTE (as current fix does), so the
return value (PlannedStmt) from standard_planner() won't have
hasModifyingCTE set correctly in the cases where the rewriter doesn't
set it correctly (and I'm not sure what strange side effects ??? that
might have).
- Although the invocation of max_parallel_hazard_walker() on a RTE
subquery will "work" in finally locating your fix's added
"CommonTableExpr" parallel-safety disabling block for commandType !=
CMD_SELECT, it looks like it potentially results in checking and
walking over a lot of other stuff within the subquery not related to
CTEs. The current fix does a more specific and efficient search for a
modifying CTE.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 24, 2021 at 4:30 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 24, 2021 at 8:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 19, 2021 at 6:56 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting a new version of the patches, with the following updates:
I am not happy with the below code changes, I think we need a better
way to deal with this.@@ -313,19 +314,35 @@ standard_planner(Query *parse, const char
*query_string, int cursorOptions,
glob->transientPlan = false;
glob->dependsOnRole = false;+ if (IsModifySupportedInParallelMode(parse->commandType) && + !parse->hasModifyingCTE) + { + /* + * FIXME + * There is a known bug in the query rewriter: re-written queries with + * a modifying CTE may not have the "hasModifyingCTE" flag set. When + * that bug is fixed, this temporary fix must be removed. + * + * Note that here we've made a fix for this problem only for a + * supported-in-parallel-mode table-modification statement (i.e. + * INSERT), but this bug exists for SELECT too. + */ + parse->hasModifyingCTE = query_has_modifying_cte(parse); + } +I understand that this is an existing bug but I am not happy with this
workaround. I feel it is better to check for modifyingCTE in
max_parallel_hazard_walker. See attached, this is atop
v18-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.Thanks, I'll try it.
I did, however, notice a few concerns with your suggested alternative fix:
- It is not restricted to INSERT (as current fix is).
So what? The Select also has a similar problem.
- It does not set parse->hasModifyingCTE (as current fix does), so the
return value (PlannedStmt) from standard_planner() won't have
hasModifyingCTE set correctly in the cases where the rewriter doesn't
set it correctly (and I'm not sure what strange side effects ??? that
might have).
Here end goal is not to set hasModifyingCTE but do let me know if you
see any problem or impact.
- Although the invocation of max_parallel_hazard_walker() on a RTE
subquery will "work" in finally locating your fix's added
"CommonTableExpr" parallel-safety disabling block for commandType !=
CMD_SELECT, it looks like it potentially results in checking and
walking over a lot of other stuff within the subquery not related to
CTEs. The current fix does a more specific and efficient search for a
modifying CTE.
I find the current fix proposed by you quite ad-hoc and don't think we
can go that way.
--
With Regards,
Amit Kapila.
On Wed, Feb 24, 2021 at 10:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Thanks, I'll try it.
I did, however, notice a few concerns with your suggested alternative fix:
- It is not restricted to INSERT (as current fix is).So what? The Select also has a similar problem.
Yes, but you're potentially now adding overhead to every
SELECT/UPDATE/DELETE with a subquery, by the added recursive checking
and walking done by the new call to max_parallel_hazard_walker().and
code block looking for a modifying CTE
And anyway I'm not sure it's really right putting in a fix for SELECT
with a modifying CTE, into a patch that adds parallel INSERT
functionality - in any case you'd need to really spell this out in
code comments, as this is at best a temporary fix that would need to
be removed whenever the query rewriter is fixed to set hasModifyingCTE
correctly.
- It does not set parse->hasModifyingCTE (as current fix does), so the
return value (PlannedStmt) from standard_planner() won't have
hasModifyingCTE set correctly in the cases where the rewriter doesn't
set it correctly (and I'm not sure what strange side effects ??? that
might have).Here end goal is not to set hasModifyingCTE but do let me know if you
see any problem or impact.
parse->hasModifyingCTE is not just used in the shortcut-test for
parallel-safety, its value is subsequently copied into the PlannedStmt
returned by standard_planner.
It's inconsistent to leave hasModifyingCTE FALSE when by the fix it
has found a modifying CTE.
Even if no existing tests detect an issue with this, PlannedStmt is
left with a bad hasModifyingCTE value in this case, so there is the
potential for something to go wrong.
- Although the invocation of max_parallel_hazard_walker() on a RTE
subquery will "work" in finally locating your fix's added
"CommonTableExpr" parallel-safety disabling block for commandType !=
CMD_SELECT, it looks like it potentially results in checking and
walking over a lot of other stuff within the subquery not related to
CTEs. The current fix does a more specific and efficient search for a
modifying CTE.I find the current fix proposed by you quite ad-hoc and don't think we
can go that way.
At least my current fix is very specific, efficient and clear in its
purpose, and suitably documented, so it is very clear when and how it
is to be removed, when the issue is fixed in the query rewriter.
Another concern with the alternative fix is that it always searches
for a modifying CTE, even when parse->hasModifyingCTE is true after
the query rewriter processing.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Feb 24, 2021 at 6:21 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 24, 2021 at 10:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Thanks, I'll try it.
I did, however, notice a few concerns with your suggested alternative fix:
- It is not restricted to INSERT (as current fix is).So what? The Select also has a similar problem.
Yes, but you're potentially now adding overhead to every
SELECT/UPDATE/DELETE with a subquery, by the added recursive checking
and walking done by the new call to max_parallel_hazard_walker().and
code block looking for a modifying CTE
Can you please share an example where it has added an overhead?
And anyway I'm not sure it's really right putting in a fix for SELECT
with a modifying CTE, into a patch that adds parallel INSERT
functionality - in any case you'd need to really spell this out in
code comments, as this is at best a temporary fix that would need to
be removed whenever the query rewriter is fixed to set hasModifyingCTE
correctly.- It does not set parse->hasModifyingCTE (as current fix does), so the
return value (PlannedStmt) from standard_planner() won't have
hasModifyingCTE set correctly in the cases where the rewriter doesn't
set it correctly (and I'm not sure what strange side effects ??? that
might have).Here end goal is not to set hasModifyingCTE but do let me know if you
see any problem or impact.parse->hasModifyingCTE is not just used in the shortcut-test for
parallel-safety, its value is subsequently copied into the PlannedStmt
returned by standard_planner.
It's inconsistent to leave hasModifyingCTE FALSE when by the fix it
has found a modifying CTE.
Even if no existing tests detect an issue with this, PlannedStmt is
left with a bad hasModifyingCTE value in this case, so there is the
potential for something to go wrong.- Although the invocation of max_parallel_hazard_walker() on a RTE
subquery will "work" in finally locating your fix's added
"CommonTableExpr" parallel-safety disabling block for commandType !=
CMD_SELECT, it looks like it potentially results in checking and
walking over a lot of other stuff within the subquery not related to
CTEs. The current fix does a more specific and efficient search for a
modifying CTE.I find the current fix proposed by you quite ad-hoc and don't think we
can go that way.At least my current fix is very specific, efficient and clear in its
purpose, and suitably documented, so it is very clear when and how it
is to be removed, when the issue is fixed in the query rewriter.
Another concern with the alternative fix is that it always searches
for a modifying CTE, even when parse->hasModifyingCTE is true after
the query rewriter processing.
There is a check in standard_planner such that if
parse->hasModifyingCTE is true then we won't try checking
parallel-safety.
--
With Regards,
Amit Kapila.
On Thu, Feb 25, 2021 at 12:19 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 24, 2021 at 6:21 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 24, 2021 at 10:38 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Thanks, I'll try it.
I did, however, notice a few concerns with your suggested alternative fix:
- It is not restricted to INSERT (as current fix is).So what? The Select also has a similar problem.
Yes, but you're potentially now adding overhead to every
SELECT/UPDATE/DELETE with a subquery, by the added recursive checking
and walking done by the new call to max_parallel_hazard_walker().and
code block looking for a modifying CTECan you please share an example where it has added an overhead?
And anyway I'm not sure it's really right putting in a fix for SELECT
with a modifying CTE, into a patch that adds parallel INSERT
functionality - in any case you'd need to really spell this out in
code comments, as this is at best a temporary fix that would need to
be removed whenever the query rewriter is fixed to set hasModifyingCTE
correctly.- It does not set parse->hasModifyingCTE (as current fix does), so the
return value (PlannedStmt) from standard_planner() won't have
hasModifyingCTE set correctly in the cases where the rewriter doesn't
set it correctly (and I'm not sure what strange side effects ??? that
might have).Here end goal is not to set hasModifyingCTE but do let me know if you
see any problem or impact.parse->hasModifyingCTE is not just used in the shortcut-test for
parallel-safety, its value is subsequently copied into the PlannedStmt
returned by standard_planner.
It's inconsistent to leave hasModifyingCTE FALSE when by the fix it
has found a modifying CTE.
Even if no existing tests detect an issue with this, PlannedStmt is
left with a bad hasModifyingCTE value in this case, so there is the
potential for something to go wrong.- Although the invocation of max_parallel_hazard_walker() on a RTE
subquery will "work" in finally locating your fix's added
"CommonTableExpr" parallel-safety disabling block for commandType !=
CMD_SELECT, it looks like it potentially results in checking and
walking over a lot of other stuff within the subquery not related to
CTEs. The current fix does a more specific and efficient search for a
modifying CTE.I find the current fix proposed by you quite ad-hoc and don't think we
can go that way.At least my current fix is very specific, efficient and clear in its
purpose, and suitably documented, so it is very clear when and how it
is to be removed, when the issue is fixed in the query rewriter.
Another concern with the alternative fix is that it always searches
for a modifying CTE, even when parse->hasModifyingCTE is true after
the query rewriter processing.There is a check in standard_planner such that if
parse->hasModifyingCTE is true then we won't try checking
parallel-safety.
OK, I retract that last concern, parallel-safety checks are skipped
when parse->hasModifyingCTE is true.
Examples of overhead will need to wait until tomorrow (and would need
to test), but seems fairly clear max_parallel_hazard_walker() first
checks parallel-unsafe functions in the node, then does numerous
node-type checks before getting to CommonTableExpr - exactly how much
extra work would depend on the SQL.
Regards,
Greg Nancarrow
Fujitsu Australia
It is quite possible what you are saying is correct but I feel that is
not this patch's fault. So, won't it better to discuss this in a
separate thread?Good use case but again, I think this can be done as a separate patch.
Agreed.
I think even the current patch offers great benefits and can be committed in PG
14, even if all my four feedback comments are left unaddressed. I just touched
on them for completeness in terms of typically expected use cases. They will
probably be able to be implemented along the current design.I think here you are talking about the third patch (Parallel Inserts).
I guess if one has run inserts parallelly from psql then also similar
behavior would have been observed. For tables, it might lead to better
results once we have the patch discussed at [1]. Actually, this needs
more investigation.[1] -
/messages/by-id/20200508072545.GA9701@telsas
oft.com
That looks interesting and worth a try.
Hi,
I test the bitmapscan with both multi-insert patch and parallel insert patch applied.
But the performance degradation and table size increased still happened in my machine.
To better analyze this issue, I did some more research on it (only applied parallel insert patch)
I add some code to track the time spent in index operation.
From the results[1]--------------parallel bitmap scan------------------ worker 0: psql:test.sql:10: INFO: insert index _bt_search_insert time:834735 psql:test.sql:10: INFO: insert index total time:1895330 psql:test.sql:10: INFO: insert tuple time:628064, we can see more workers will bring more cost in _bt_search_insert() in each worker.
After debugged, the most cost part is the following:
-----
/* drop the read lock on the page, then acquire one on its child */
*bufP = _bt_relandgetbuf(rel, *bufP, child, page_access);
-----
It seems the order of parallel bitmap scan's result will result in more lock time in parallel insert.
[1]: --------------parallel bitmap scan------------------ worker 0: psql:test.sql:10: INFO: insert index _bt_search_insert time:834735 psql:test.sql:10: INFO: insert index total time:1895330 psql:test.sql:10: INFO: insert tuple time:628064
worker 0:
psql:test.sql:10: INFO: insert index _bt_search_insert time:834735
psql:test.sql:10: INFO: insert index total time:1895330
psql:test.sql:10: INFO: insert tuple time:628064
worker 2:
psql:test.sql:10: INFO: insert index _bt_search_insert time:1552242
psql:test.sql:10: INFO: insert index total time:2374741
psql:test.sql:10: INFO: insert tuple time:314571
worker 4:
psql:test.sql:10: INFO: insert index _bt_search_insert time:2496424
psql:test.sql:10: INFO: insert index total time:3016150
psql:test.sql:10: INFO: insert tuple time:211741
----------------------------
Based on above, I tried to change the order of results that bitmapscan return.
In the original test, we prepare data in order (like: generate_series(1,10000,1)),
If we change the order we insert the data in the source table, the performance degradation will not always happen[2]------------------------------------------------------- Worker 0: Execution Time: 37028.006 ms Worker 2: Execution Time: 11355.153 ms Worker 4: Execution Time: 9273.398 ms --------------------------------------------------------.
And table size difference will be small.
-------------------out of order source table-----------------------------
insert into x(a,b,c) select i,i+1,i+2 from generate_series(1,600000000) as t(i) order by random();
----------------------------------------------------------------------------
Test results when source table out of order(using bitmap heap scan):
[2]: ------------------------------------------------------- Worker 0: Execution Time: 37028.006 ms Worker 2: Execution Time: 11355.153 ms Worker 4: Execution Time: 9273.398 ms --------------------------------------------------------
Worker 0:
Execution Time: 37028.006 ms
Worker 2:
Execution Time: 11355.153 ms
Worker 4:
Execution Time: 9273.398 ms
--------------------------------------------------------
So, this performance degradation issue seems related on the order of the data in the source table.
It does not always happen. Do we need to do some specific fix for it ?
For multi-insert, I guess the reason why it does not solve the performance problem is that we do not actually have a api for multi-index insert,
Like the api for tableam rd_tableam->multi_insert(), so we still execute ExecInsertIndexTuples in a loop for the multi index insert.
I plan to do some more test for multi-insert and parallel insert with out of order source table.
Best regards,
houzj
On Wed, Feb 24, 2021 at 6:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 24, 2021 at 2:14 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Wed, Feb 24, 2021 at 3:12 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 24, 2021 at 8:41 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Feb 23, 2021 at 10:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
But the non-parallel plan was chosen (instead of a parallel plan)
because of parallel-safety checks on the partitions, which found
attributes of the partitions which weren't parallel-safe.
So it's not so clear to me that the dependency doesn't exist - the
non-parallel plan does in fact depend on the state of the partitions.Hmm, I think that is not what we can consider as a dependency.
Then if it's not a dependency, then we shouldn't have to check the
attributes of the partitions for parallel-safety, to determine whether
we must use a non-parallel plan (or can use a parallel plan).
Except, of course, we do have to ...I don't think the plan-dependency and checking for parallel-safety are
directly related.That is certainly not my understanding. Why do you think that they are
not directly related?
This whole issue came about because Amit L pointed out that there is a
need to add partition OIDs as plan-dependencies BECAUSE the checking
for parallel-safety and plan-dependency are related - since now, for
Parallel INSERT, we're executing extra parallel-safety checks that
check partition properties, so the resultant plan is dependent on the
partitions and their properties.He has pointed out an issue when the plan is parallel and you can see
in that example that it fails if we didn't invalidate it. For
non-parallel plans, there won't be any such issue.
Yes. I checked around a bit (code and -hackers archive [1]Plan invalidation design: /messages/by-id/20244.1171734513@sss.pgh.pa.us) and came
away with the impression that there do not appear to be any set rules
for deciding which object changes to send an invalidation message for
(sending side: ddl, vacuum/analyze) and which items of a Query or a
Plan to track changes for (receiving side: planner, plancache). One
could say the foremost rule is to avoid broken cached plans and only
in some really obvious cases do the thing that produces a better plan
[2]: ...contradicts what I said before, but I found this comment in DefineIndex():
to go for the latter probably involves some cost-benefit analysis,
something we can probably revisit.
I don't think we're compromising by not adding the partition OIDs when
the insert plan is not parallel, but the benefits of adding them in
all cases are not so clear cut that maybe it's not worth it.
--
Amit Langote
EDB: http://www.enterprisedb.com
[1]: Plan invalidation design: /messages/by-id/20244.1171734513@sss.pgh.pa.us
/messages/by-id/20244.1171734513@sss.pgh.pa.us
[2]: ...contradicts what I said before, but I found this comment in DefineIndex():
DefineIndex():
/*
* The pg_index update will cause backends (including this one) to update
* relcache entries for the index itself, but we should also send a
* relcache inval on the parent table to force replanning of cached plans.
* Otherwise existing sessions might fail to use the new index where it
* would be useful. (Note that our earlier commits did not create reasons
* to replan; so relcache flush on the index itself was sufficient.)
*/
CacheInvalidateRelcacheByRelid(heaprelid.relId);
So this invalidates any plans referencing the index's parent relation
to trigger replanning so as to take the index into account. The old
plans would not really be "unrunnable" without the index though.
Posting a new version of the patches, with the following updates:
- Added parallel-safety check for index predicate and added extra
tests for this.
- Removed fix for query rewriter hasModifyingCTE bug and instead
handle this by detecting a modifying CTE within the parallel-safety
checks (and returning UNSAFE if one is found).
- Updated registration of parallel-safety-checked partition OIDs as
plan dependencies (only do it for parallel plans).
- Integrated parallel_dml patch.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v19-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v19-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 69f9b6db680ae04041655332cead0f05e0187c3b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 26 Feb 2021 11:23:22 +1100
Subject: [PATCH v19 1/5] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or predicate,
column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/optimizer/plan/planner.c | 26 +-
src/backend/optimizer/plan/setrefs.c | 20 +
src/backend/optimizer/util/clauses.c | 525 ++++++++++++++++++++++++++-
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/optimizer/clauses.h | 4 +-
8 files changed, 601 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df6b8..e1d16fc1cf 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56bcaf..e60b484f47 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -247,7 +247,6 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
/*****************************************************************************
*
* Query optimizer entry point
@@ -305,6 +304,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +334,15 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index c3c36be13e..0b58d23ff2 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,10 @@
*/
#include "postgres.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +27,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -260,6 +265,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If modifying a partitioned table, add its parallel-safety-checked
+ * partitions too to glob->relationOids, to register them as plan
+ * dependencies. This is only really needed in the case of a parallel
+ * plan, so that if parallel-unsafe properties are subsequently defined
+ * on the partitions, the cached parallel plan will be invalidated and
+ * a non-parallel plan will be generated.
+ */
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+ }
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..d1ec5a8e9c 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +170,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +566,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +609,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -687,6 +716,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with
+ * a modifying CTE does not have that flag set, due to a bug in the
+ * query rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -757,6 +807,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +840,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +851,460 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *)ii_Predicate, context))
+ {
+ found_max_hazard = true;
+ }
+ }
+ }
+
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+
+ /* Record the partition as a potential plan dependency. */
+ glob->partitionOids =
+ lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only parallel INSERT is currently supported,
+ * only command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan
+ * of the query tree in standard_planner().
+ * Note that even though column defaults may be specified separately for
+ * each partition in a partitioned table, a partition's default value is
+ * not applied when inserting a tuple through a partitioned table.
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_possible_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_possible_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ Assert(IsModifySupportedInParallelMode(parse->commandType));
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index 0ec93e648c..6eabab4224 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..e2f0fe3aa6 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,7 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
+extern bool is_parallel_possible_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v19-0005-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v19-0005-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From a4134484c1511a2ca7d043116d47da2ceb56a830 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 23:39:40 +1100
Subject: [PATCH v19 5/5] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++-
src/test/regress/expected/insert_parallel.out | 142 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 +--
3 files changed, 158 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 62e86cc214..868282437e 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -122,14 +122,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -165,7 +165,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -212,15 +212,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -305,9 +305,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -344,9 +344,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -386,9 +386,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -436,9 +436,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -471,9 +471,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -507,9 +507,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -545,9 +545,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -585,9 +585,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -659,21 +659,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -702,7 +702,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -738,9 +738,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -801,14 +801,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -857,7 +857,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -886,14 +886,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -969,9 +969,9 @@ alter table parttable1 set (parallel_dml_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1033,9 +1033,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1072,7 +1072,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -1093,11 +1093,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1138,7 +1138,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1195,13 +1195,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1228,9 +1228,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 55ba2e2b5b..643de1b129 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -136,7 +136,7 @@ alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -148,7 +148,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -161,7 +161,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -305,14 +305,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -320,7 +320,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -373,7 +373,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -389,7 +389,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -397,7 +397,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -496,7 +496,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -546,7 +546,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -592,7 +592,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
v19-0003-Add-new-parallel-dml-GUC-and-table-options.patchapplication/octet-stream; name=v19-0003-Add-new-parallel-dml-GUC-and-table-options.patchDownload
From 5715174337b8981a67b00b4b4de0e778b1bf7985 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 23:51:14 +1100
Subject: [PATCH v19 3/5] Add new GUC option "enable_parallel_dml" (boolean)
and table option "parallel_dml_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_dml" is
added, to allow parallel DML to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel dml with finer granularity, to enable/disable the use of parallel
dml for specific tables.
The new table option "parallel_dml_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++
src/backend/access/common/reloptions.c | 25 ++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 ++++++++--
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b5718fc136..3fba832711 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5061,6 +5061,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-dml" xreflabel="enable_parallel_dml">
+ <term><varname>enable_parallel_dml</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_dml</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ table-modification commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for table-modification. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ table-modification.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..ecb047021d 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_dml_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..5d8f842ddd 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-dml-enabled" xreflabel="parallel_dml_enabled">
+ <term><literal>parallel_dml_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_dml_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel DML for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-dml"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for table-modification. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel DML for this table.
+ Note that if the target table of the parallel DML is partitioned, the
+ <literal>parallel_dml_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..938131a29a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_dml_enabled",
+ "Enables \"parallel dml\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_dml_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_dml_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index aab06c7d21..bdbf9b1e41 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_dml = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index d1ec5a8e9c..0fbfa7f5a6 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1257,8 +1257,10 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
* table-modification statement.
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_dml is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_dml_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1269,11 +1271,16 @@ bool
is_parallel_possible_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
Assert(IsModifySupportedInParallelMode(parse->commandType));
+ if (!enable_parallel_dml)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1303,7 +1310,23 @@ is_parallel_possible_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelDML(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d626731723..27b0c0c099 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2048,6 +2048,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_dml", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for table-modification commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_dml,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..5c1266ef39 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_dml = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..e7e4af795e 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_dml_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index ed2e4af4be..a566562a5c 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_dml;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..7dce70ecea 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelDML
+ * Returns the relation's parallel_dml_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelDML(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_dml_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_dml_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 9705de7622..62e86cc214 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -85,12 +85,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -914,9 +944,28 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..6c25bc1e2f 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
+ enable_parallel_dml | off
enable_parallel_hash | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index f881f3d791..55ba2e2b5b 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -105,13 +105,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
@@ -394,10 +415,25 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.27.0
v19-0004-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v19-0004-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 13e0c5371c2458878cbf38521a04751b909739f6 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 10 Feb 2021 14:50:40 +1100
Subject: [PATCH v19 4/5] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 28 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 511 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 9c1d590dc7..96a0936c8e 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2147,10 +2150,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e1d16fc1cf..d6462d236c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index bdbf9b1e41..778f71b98a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 6c8305c977..f04fa7e0b2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -339,7 +339,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index e60b484f47..18610c2345 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1794,7 +1798,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1842,6 +1959,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2282,96 +2400,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2390,6 +2445,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7559,7 +7621,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 0b58d23ff2..9bafd5cb1d 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -257,6 +257,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -322,7 +323,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1070,6 +1073,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 9be0c4a6af..a47a88a249 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3524,6 +3524,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3534,10 +3535,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3554,47 +3555,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3610,6 +3586,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 943931f65d..86e811f974 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index a566562a5c..4af0beb9c8 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 8dfc36a4e1..273d92f877 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -263,7 +263,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 7dce70ecea..1197ae24d3 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -598,15 +599,15 @@ typedef struct PartitionedOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v19-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v19-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 717bbe52ad78dbce8e09c0e2af188e6e6e089512 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 17:28:59 +1100
Subject: [PATCH v19 2/5] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1292 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 656 +++++++++
5 files changed, 1953 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..9705de7622
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1292 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+insert into names9 select * from names;
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index 12bb67e491..37dc86359b 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 59b416fd80..4504af73cb 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -147,6 +147,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..f881f3d791
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,656 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
Hi Greg,
Replying to an earlier email in the thread because I think I found a
problem relevant to the topic that was brought up.
On Wed, Feb 17, 2021 at 10:34 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 17, 2021 at 10:44 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Is the use of "table_close(rel, NoLock)'' intentional? That will keep
the lock (lockmode) until end-of-transaction.I think we always keep any locks on relations that are involved in a
plan until end-of-transaction. What if a partition is changed in an
unsafe manner between being considered safe for parallel insertion and
actually performing the parallel insert?
I realized that there is a race condition that will allow a concurrent
backend to invalidate a parallel plan (for insert into a partitioned
table) at a point in time when it's too late for plancache.c to detect
it. It has to do with how plancache.c locks the relations affected by
a cached query and its cached plan. Specifically,
AcquireExecutorLocks(), which locks the relations that need to be
locked before the plan could be considered safe to execute, does not
notice the partitions that would have been checked for parallel safety
when the plan was made. Given that AcquireExecutorLocks() only loops
over PlannedStmt.rtable and locks exactly the relations found there,
partitions are not locked. This means that a concurrent backend can,
for example, add an unsafe trigger to a partition before a parallel
worker inserts into it only to fail when it does.
Steps to reproduce (tried with v19 set of patches):
drop table if exists rp, foo;
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
set plan_cache_mode to force_generic_plan;
set enable_parallel_dml to on;
deallocate all;
prepare q as insert into rp select * from foo where a%2 = 0;
explain analyze execute q;
At this point, attach a debugger to this backend and set a breakpoint
in AcquireExecutorLocks() and execute q again:
-- will execute the cached plan
explain analyze execute q;
Breakpoint will be hit. Continue till return from the function and
stop. Start another backend and execute this:
-- will go through, because no lock still taken on the partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();
Back to the debugger, hit continue to let the plan execute. You
should see this error:
ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement parallel worker
The attached patch fixes this, although I am starting to have second
thoughts about how we're tracking partitions in this patch. Wondering
if we should bite the bullet and add partitions into the main range
table instead of tracking them separately in partitionOids, which
might result in a cleaner patch overall.
--
Amit Langote
EDB: http://www.enterprisedb.com
Attachments:
AcquireExecutorLocks-partition-insert.patchapplication/octet-stream; name=AcquireExecutorLocks-partition-insert.patchDownload
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 65bbc18ecb..d666e860da 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index f5dcedf6e8..5c2cdc697b 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2211,6 +2212,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 4388aae71d..8962b37e2d 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 18610c2345..85659acd75 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -527,6 +527,9 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+ /* For AcquireExecutorLocks(). */
+ if (IsModifySupportedInParallelMode(parse->commandType))
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a0950489d..75d1088901 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,15 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL &&
+ plannedstmt->parallelModeNeeded)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 43160439f0..8b3923c77f 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
I add some code to track the time spent in index operation.
From the results[1], we can see more workers will bring more cost in
_bt_search_insert() in each worker.
After debugged, the most cost part is the following:
-----
/* drop the read lock on the page, then acquire one on its child
*/
*bufP = _bt_relandgetbuf(rel, *bufP, child, page_access);
-----It seems the order of parallel bitmap scan's result will result in more lock time in
parallel insert.
[1]---------------parallel bitmap scan------------------ worker 0:
psql:test.sql:10: INFO: insert index _bt_search_insert time:834735
psql:test.sql:10: INFO: insert index total time:1895330
psql:test.sql:10: INFO: insert tuple time:628064worker 2:
psql:test.sql:10: INFO: insert index _bt_search_insert time:1552242
psql:test.sql:10: INFO: insert index total time:2374741
psql:test.sql:10: INFO: insert tuple time:314571worker 4:
psql:test.sql:10: INFO: insert index _bt_search_insert time:2496424
psql:test.sql:10: INFO: insert index total time:3016150
psql:test.sql:10: INFO: insert tuple time:211741
----------------------------Based on above, I tried to change the order of results that bitmapscan return.
In the original test, we prepare data in order (like: generate_series(1,10000,1)), If
we change the order we insert the data in the source table, the performance
degradation will not always happen[2].
And table size difference will be small.-------------------out of order source table-----------------------------
insert into x(a,b,c) select i,i+1,i+2 from generate_series(1,600000000) as t(i)
order by random();
----------------------------------------------------------------------------Test results when source table out of order(using bitmap heap scan):
[2]--------------------------------------------------------
Worker 0:
Execution Time: 37028.006 ms
Worker 2:
Execution Time: 11355.153 ms
Worker 4:
Execution Time: 9273.398 ms
--------------------------------------------------------So, this performance degradation issue seems related on the order of the data
in the source table.
It does not always happen. Do we need to do some specific fix for it ?
After doing some more tests on it (performance degradation will not happen when source table is out of order).
I think we can say the performance degradation is related to the order of the data in source table.
Also , In master branch, I found the order of data in source table will not affect the planner when generating plan(for select part).
As we can see from [1]----------Source table data in order and max_parallel_workers_per_gather = 0----------- Bitmap Heap Scan on public.x (cost=22999.40..4991850.30 rows=91002 width=4) (actual time=45.445..201.322 rows=579999 loops=1) Output: a Recheck Cond: ((x.a < 80000) OR (x.a > 199000000)) Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000))) Rows Removed by Filter: 500000 Heap Blocks: exact=5840 -> BitmapOr (cost=22999.40..22999.40 rows=1242768 width=0) (actual time=44.622..44.637 rows=0 loops=1) -> Bitmap Index Scan on x_a_idx (cost=0.00..1575.70 rows=85217 width=0) (actual time=3.622..3.634 rows=79999 loops=1) Index Cond: (x.a < 80000) -> Bitmap Index Scan on x_a_idx (cost=0.00..21378.20 rows=1157551 width=0) (actual time=40.998..40.998 rows=1000000 loops=1) Index Cond: (x.a > 199000000) Planning Time: 0.199 ms Execution Time: 232.327 ms[2]----------Source table data in order and max_parallel_workers_per_gather = 4----------- Gather (cost=1000.00..2091183.08 rows=91002 width=4) (actual time=0.594..4216.197 rows=579999 loops=1) Output: a Workers Planned: 4 Workers Launched: 4 -> Parallel Seq Scan on public.x (cost=0.00..2081082.88 rows=22750 width=4) (actual time=0.087..4197.570 rows=116000 loops=5) Output: a Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000))) Rows Removed by Filter: 39884000 Worker 0: actual time=0.083..4219.087 rows=96574 loops=1 Worker 1: actual time=0.076..4201.339 rows=195354 loops=1 Worker 2: actual time=0.096..4218.637 rows=96474 loops=1 Worker 3: actual time=0.118..4205.825 rows=96847 loops=1 Planning Time: 0.175 ms Execution Time: 4243.089 ms (14 rows), If source table's data is in order, when set max_parallel_workers_per_gather to 4,
the planner will choose parallel seqscan here but it is actually slower than serial bitmapscan.
If data in source table is out of order, the performance degradation will not happen again[3]----------Source table data out of order and max_parallel_workers_per_gather = 0----------- Bitmap Heap Scan on public.x2 (cost=19815.15..4953678.20 rows=81178 width=4) (actual time=263.640..15653.251 rows=579999 loops=1) Output: a Recheck Cond: ((x2.a < 80000) OR (x2.a > 199000000)) Rows Removed by Index Recheck: 115394602 Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000))) Rows Removed by Filter: 500000 Heap Blocks: exact=55343 lossy=629270 -> BitmapOr (cost=19815.15..19815.15 rows=1070588 width=0) (actual time=248.701..248.715 rows=0 loops=1) -> Bitmap Index Scan on x2_a_idx (cost=0.00..1408.13 rows=76208 width=0) (actual time=18.116..18.117 rows=79999 loops=1) Index Cond: (x2.a < 80000) -> Bitmap Index Scan on x2_a_idx (cost=0.00..18366.43 rows=994381 width=0) (actual time=230.581..230.582 rows=1000000 loops=1) Index Cond: (x2.a > 199000000) Planning Time: 0.530 ms Execution Time: 15700.488 ms[4]----------Source table data out of order and max_parallel_workers_per_gather = 4----------- Gather (cost=1000.00..2090199.80 rows=81178 width=4) (actual time=0.674..5154.622 rows=579999 loops=1) Output: a Workers Planned: 4 Workers Launched: 4 -> Parallel Seq Scan on public.x2 (cost=0.00..2081082.00 rows=20294 width=4) (actual time=0.124..5112.635 rows=116000 loops=5) Output: a Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000))) Rows Removed by Filter: 39884000 Worker 0: actual time=0.220..5136.219 rows=107646 loops=1 Worker 1: actual time=0.170..5133.046 rows=114824 loops=1 Worker 2: actual time=0.127..5128.256 rows=115010 loops=1 Worker 3: actual time=0.066..5133.022 rows=120061 loops=1 Planning Time: 0.194 ms Execution Time: 5187.682 ms.
So, the order of data 's influence seems a normal phenomenon, I think it seems we do not need to do anything about it (currently).
It seems better to mark it as todo which we can improve this in the future.
(Since the performance degradation in parallel bitmap is because of the lock in _bt_search, It will not always happen
when the target table already have data, less race condition will happened when parallel insert into a evenly distributed btree).
Test result with sql: "postgres=# explain analyze verbose select a from x where a<80000 or (a%2=0 and a>199000000);"
[1]: ----------Source table data in order and max_parallel_workers_per_gather = 0----------- Bitmap Heap Scan on public.x (cost=22999.40..4991850.30 rows=91002 width=4) (actual time=45.445..201.322 rows=579999 loops=1) Output: a Recheck Cond: ((x.a < 80000) OR (x.a > 199000000)) Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000))) Rows Removed by Filter: 500000 Heap Blocks: exact=5840 -> BitmapOr (cost=22999.40..22999.40 rows=1242768 width=0) (actual time=44.622..44.637 rows=0 loops=1) -> Bitmap Index Scan on x_a_idx (cost=0.00..1575.70 rows=85217 width=0) (actual time=3.622..3.634 rows=79999 loops=1) Index Cond: (x.a < 80000) -> Bitmap Index Scan on x_a_idx (cost=0.00..21378.20 rows=1157551 width=0) (actual time=40.998..40.998 rows=1000000 loops=1) Index Cond: (x.a > 199000000) Planning Time: 0.199 ms Execution Time: 232.327 ms
Bitmap Heap Scan on public.x (cost=22999.40..4991850.30 rows=91002 width=4) (actual time=45.445..201.322 rows=579999 loops=1)
Output: a
Recheck Cond: ((x.a < 80000) OR (x.a > 199000000))
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000)))
Rows Removed by Filter: 500000
Heap Blocks: exact=5840
-> BitmapOr (cost=22999.40..22999.40 rows=1242768 width=0) (actual time=44.622..44.637 rows=0 loops=1)
-> Bitmap Index Scan on x_a_idx (cost=0.00..1575.70 rows=85217 width=0) (actual time=3.622..3.634 rows=79999 loops=1)
Index Cond: (x.a < 80000)
-> Bitmap Index Scan on x_a_idx (cost=0.00..21378.20 rows=1157551 width=0) (actual time=40.998..40.998 rows=1000000 loops=1)
Index Cond: (x.a > 199000000)
Planning Time: 0.199 ms
Execution Time: 232.327 ms
[2]: ----------Source table data in order and max_parallel_workers_per_gather = 4----------- Gather (cost=1000.00..2091183.08 rows=91002 width=4) (actual time=0.594..4216.197 rows=579999 loops=1) Output: a Workers Planned: 4 Workers Launched: 4 -> Parallel Seq Scan on public.x (cost=0.00..2081082.88 rows=22750 width=4) (actual time=0.087..4197.570 rows=116000 loops=5) Output: a Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000))) Rows Removed by Filter: 39884000 Worker 0: actual time=0.083..4219.087 rows=96574 loops=1 Worker 1: actual time=0.076..4201.339 rows=195354 loops=1 Worker 2: actual time=0.096..4218.637 rows=96474 loops=1 Worker 3: actual time=0.118..4205.825 rows=96847 loops=1 Planning Time: 0.175 ms Execution Time: 4243.089 ms (14 rows)
Gather (cost=1000.00..2091183.08 rows=91002 width=4) (actual time=0.594..4216.197 rows=579999 loops=1)
Output: a
Workers Planned: 4
Workers Launched: 4
-> Parallel Seq Scan on public.x (cost=0.00..2081082.88 rows=22750 width=4) (actual time=0.087..4197.570 rows=116000 loops=5)
Output: a
Filter: ((x.a < 80000) OR (((x.a % 2) = 0) AND (x.a > 199000000)))
Rows Removed by Filter: 39884000
Worker 0: actual time=0.083..4219.087 rows=96574 loops=1
Worker 1: actual time=0.076..4201.339 rows=195354 loops=1
Worker 2: actual time=0.096..4218.637 rows=96474 loops=1
Worker 3: actual time=0.118..4205.825 rows=96847 loops=1
Planning Time: 0.175 ms
Execution Time: 4243.089 ms
(14 rows)
[3]: ----------Source table data out of order and max_parallel_workers_per_gather = 0----------- Bitmap Heap Scan on public.x2 (cost=19815.15..4953678.20 rows=81178 width=4) (actual time=263.640..15653.251 rows=579999 loops=1) Output: a Recheck Cond: ((x2.a < 80000) OR (x2.a > 199000000)) Rows Removed by Index Recheck: 115394602 Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000))) Rows Removed by Filter: 500000 Heap Blocks: exact=55343 lossy=629270 -> BitmapOr (cost=19815.15..19815.15 rows=1070588 width=0) (actual time=248.701..248.715 rows=0 loops=1) -> Bitmap Index Scan on x2_a_idx (cost=0.00..1408.13 rows=76208 width=0) (actual time=18.116..18.117 rows=79999 loops=1) Index Cond: (x2.a < 80000) -> Bitmap Index Scan on x2_a_idx (cost=0.00..18366.43 rows=994381 width=0) (actual time=230.581..230.582 rows=1000000 loops=1) Index Cond: (x2.a > 199000000) Planning Time: 0.530 ms Execution Time: 15700.488 ms
Bitmap Heap Scan on public.x2 (cost=19815.15..4953678.20 rows=81178 width=4) (actual time=263.640..15653.251 rows=579999 loops=1)
Output: a
Recheck Cond: ((x2.a < 80000) OR (x2.a > 199000000))
Rows Removed by Index Recheck: 115394602
Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000)))
Rows Removed by Filter: 500000
Heap Blocks: exact=55343 lossy=629270
-> BitmapOr (cost=19815.15..19815.15 rows=1070588 width=0) (actual time=248.701..248.715 rows=0 loops=1)
-> Bitmap Index Scan on x2_a_idx (cost=0.00..1408.13 rows=76208 width=0) (actual time=18.116..18.117 rows=79999 loops=1)
Index Cond: (x2.a < 80000)
-> Bitmap Index Scan on x2_a_idx (cost=0.00..18366.43 rows=994381 width=0) (actual time=230.581..230.582 rows=1000000 loops=1)
Index Cond: (x2.a > 199000000)
Planning Time: 0.530 ms
Execution Time: 15700.488 ms
[4]: ----------Source table data out of order and max_parallel_workers_per_gather = 4----------- Gather (cost=1000.00..2090199.80 rows=81178 width=4) (actual time=0.674..5154.622 rows=579999 loops=1) Output: a Workers Planned: 4 Workers Launched: 4 -> Parallel Seq Scan on public.x2 (cost=0.00..2081082.00 rows=20294 width=4) (actual time=0.124..5112.635 rows=116000 loops=5) Output: a Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000))) Rows Removed by Filter: 39884000 Worker 0: actual time=0.220..5136.219 rows=107646 loops=1 Worker 1: actual time=0.170..5133.046 rows=114824 loops=1 Worker 2: actual time=0.127..5128.256 rows=115010 loops=1 Worker 3: actual time=0.066..5133.022 rows=120061 loops=1 Planning Time: 0.194 ms Execution Time: 5187.682 ms
Gather (cost=1000.00..2090199.80 rows=81178 width=4) (actual time=0.674..5154.622 rows=579999 loops=1)
Output: a
Workers Planned: 4
Workers Launched: 4
-> Parallel Seq Scan on public.x2 (cost=0.00..2081082.00 rows=20294 width=4) (actual time=0.124..5112.635 rows=116000 loops=5)
Output: a
Filter: ((x2.a < 80000) OR (((x2.a % 2) = 0) AND (x2.a > 199000000)))
Rows Removed by Filter: 39884000
Worker 0: actual time=0.220..5136.219 rows=107646 loops=1
Worker 1: actual time=0.170..5133.046 rows=114824 loops=1
Worker 2: actual time=0.127..5128.256 rows=115010 loops=1
Worker 3: actual time=0.066..5133.022 rows=120061 loops=1
Planning Time: 0.194 ms
Execution Time: 5187.682 ms
Best regards,
houzj
On Fri, Feb 26, 2021 at 4:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
Hi Greg,
Replying to an earlier email in the thread because I think I found a
problem relevant to the topic that was brought up.On Wed, Feb 17, 2021 at 10:34 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Wed, Feb 17, 2021 at 10:44 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Is the use of "table_close(rel, NoLock)'' intentional? That will keep
the lock (lockmode) until end-of-transaction.I think we always keep any locks on relations that are involved in a
plan until end-of-transaction. What if a partition is changed in an
unsafe manner between being considered safe for parallel insertion and
actually performing the parallel insert?I realized that there is a race condition that will allow a concurrent
backend to invalidate a parallel plan (for insert into a partitioned
table) at a point in time when it's too late for plancache.c to detect
it. It has to do with how plancache.c locks the relations affected by
a cached query and its cached plan. Specifically,
AcquireExecutorLocks(), which locks the relations that need to be
locked before the plan could be considered safe to execute, does not
notice the partitions that would have been checked for parallel safety
when the plan was made. Given that AcquireExecutorLocks() only loops
over PlannedStmt.rtable and locks exactly the relations found there,
partitions are not locked. This means that a concurrent backend can,
for example, add an unsafe trigger to a partition before a parallel
worker inserts into it only to fail when it does.Steps to reproduce (tried with v19 set of patches):
drop table if exists rp, foo;
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
set plan_cache_mode to force_generic_plan;
set enable_parallel_dml to on;
deallocate all;
prepare q as insert into rp select * from foo where a%2 = 0;
explain analyze execute q;At this point, attach a debugger to this backend and set a breakpoint
in AcquireExecutorLocks() and execute q again:-- will execute the cached plan
explain analyze execute q;Breakpoint will be hit. Continue till return from the function and
stop. Start another backend and execute this:-- will go through, because no lock still taken on the partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();Back to the debugger, hit continue to let the plan execute. You
should see this error:ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement parallel workerThe attached patch fixes this, although I am starting to have second
thoughts about how we're tracking partitions in this patch. Wondering
if we should bite the bullet and add partitions into the main range
table instead of tracking them separately in partitionOids, which
might result in a cleaner patch overall.
Thanks Amit,
I was able to reproduce the problem using your instructions (though I
found I had to run that explain an extra time, in order to hit the
breakpoint).
Also, I can confirm that the problem doesn't occur after application
of your patch.
I'll leave it to your better judgement as to what to do next - if you
feel the current tracking method is not sufficient
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Feb 26, 2021 at 3:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 26, 2021 at 4:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
The attached patch fixes this, although I am starting to have second
thoughts about how we're tracking partitions in this patch. Wondering
if we should bite the bullet and add partitions into the main range
table instead of tracking them separately in partitionOids, which
might result in a cleaner patch overall.Thanks Amit,
I was able to reproduce the problem using your instructions (though I
found I had to run that explain an extra time, in order to hit the
breakpoint).
Also, I can confirm that the problem doesn't occur after application
of your patch.
I'll leave it to your better judgement as to what to do next - if you
feel the current tracking method is not sufficient
Just to be clear, I think the tracking method added by the patch is
sufficient AFAICS for the problems we were able to discover. The
concern I was trying to express is that we seem to be duct-taping
holes in our earlier chosen design to track partitions separately from
the range table. If we had decided to add partitions to the range
table as "extra" target relations from the get-go, both the issues I
mentioned with cached plans -- partitions not being counted as a
dependency and partitions not being locked before execution -- would
have been prevented. I haven't fully grasped how invasive that design
would be, but it sure sounds like it would be a bit more robust.
--
Amit Langote
EDB: http://www.enterprisedb.com
From: Hou, Zhijie/侯 志杰 <houzj.fnst@fujitsu.com>
After doing some more tests on it (performance degradation will not happen
when source table is out of order).
I think we can say the performance degradation is related to the order of the
data in source table.
...
So, the order of data 's influence seems a normal phenomenon, I think it seems
we do not need to do anything about it (currently).
It seems better to mark it as todo which we can improve this in the future.(Since the performance degradation in parallel bitmap is because of the lock in
_bt_search, It will not always happen when the target table already have data,
less race condition will happened when parallel insert into a evenly distributed
btree).
I think so, too. The slowness of parallel insert operation due to index page contention, and index bloat, would occur depending on the order of the index key values of source records.
I guess other DBMSs exhibit similar phenomenon, but I couldn't find such description in the manual, whitepapers, or several books on Oracle. One relevant excerpt is the following. This is about parallel index build. Oracle tries to minimize page contention and index bloat. This is completely my guess, but they may do similar things in parallel INSERT SELECT, because Oracle holds an exclusive lock on the target table. SQL Server also acquires an exclusive lock. Maybe we can provide an option to do so in the future.
https://docs.oracle.com/en/database/oracle/oracle-database/21/vldbg/parallel-exec-tips.html#GUID-08A08783-C243-4872-AFFA-56B603F1F0F5
--------------------------------------------------
Optimizing Performance by Creating Indexes in Parallel
...
Multiple processes can work simultaneously to create an index. By dividing the work necessary to create an index among multiple server processes, Oracle Database can create the index more quickly than if a single server process created the index serially.
Parallel index creation works in much the same way as a table scan with an ORDER BY clause. The table is randomly sampled and a set of index keys is found that equally divides the index into the same number of pieces as the DOP. A first set of query processes scans the table, extracts key-rowid pairs, and sends each pair to a process in a second set of query processes based on a key. Each process in the second set sorts the keys and builds an index in the usual fashion. After all index pieces are built, the parallel execution coordinator simply concatenates the pieces (which are ordered) to form the final index.
...
When creating an index in parallel, the STORAGE clause refers to the storage of each of the subindexes created by the query server processes. Therefore, an index created with an INITIAL value of 5 MB and a DOP of 12 consumes at least 60 MB of storage during index creation because each process starts with an extent of 5 MB. When the query coordinator process combines the sorted subindexes, some extents might be trimmed, and the resulting index might be smaller than the requested 60 MB.
--------------------------------------------------
IIRC, the current patch showd nice performance improvement in some (many?) patterns. So, I think it can be committed in PG 14, when it has addressed the plan cache issue that Amit Langote-san posed. I remember the following issues/comments are pending, but they are not blockers:
1. Insert operation is run serially when the target table has a foreign key, sequence or identity column.
This can be added later based on the current design without requiring rework. That is, the current patch leaves no debt. (Personally, foreign key and sequence support will also be wanted in PG 14. We may try them in the last CF once the current patch is likely to be committable.)
2. There's a plausible reason for the performance variation and index bloat with the bitmap scan case.
Ideally, we want to come up with a solution that can be incorporated in PG 15.
Or, it may be one conclusion that we can't prevent performance degradation in all cases. That may be one hidden reason why Oracle and SQL Server doesn't enable parallel DML by default.
We can advise the user in the manual that parallel DML is not always faster than serial operation so he should test performance by enabling and disabling parallel DML. Also, maybe we should honestly state that indexes can get a bit bigger after parallel insert than after serial insert, and advise the user to do REINDEX CONCURRENTLY if necessary.
3. The total time of parallel execution can get longer because of unbalanced work distribution among parallel workers.
This seems to be an existing problem, so we can pursue the improvement later, hopefully before the release of PG 14.
Does anyone see any problem with committing the current patch (after polishing it)?
Regards
Takayuki Tsunakawa
From: Tsunakawa, Takayuki/綱川 貴之 <tsunakawa.takay@fujitsu.com>
the current patch showd nice performance improvement in some (many?) patterns.
So, I think it can be committed in PG 14, when it has addressed the plan cache issue that Amit Langote-san posed.
Agreed.
I summarized my test results for the current patch(V18) in the attached file(Please use VSCode or Notepad++ to open it, the context is a bit long).
As you can see, the performance is good in many patterns(Please refer to my test script, test NO in text is correspond to number in sql file).
If you have any question on my test, please feel free to ask.
Regards,
Tang
On Fri, Feb 26, 2021 at 5:50 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Fri, Feb 26, 2021 at 3:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 26, 2021 at 4:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
The attached patch fixes this, although I am starting to have second
thoughts about how we're tracking partitions in this patch. Wondering
if we should bite the bullet and add partitions into the main range
table instead of tracking them separately in partitionOids, which
might result in a cleaner patch overall.Thanks Amit,
I was able to reproduce the problem using your instructions (though I
found I had to run that explain an extra time, in order to hit the
breakpoint).
Also, I can confirm that the problem doesn't occur after application
of your patch.
I'll leave it to your better judgement as to what to do next - if you
feel the current tracking method is not sufficientJust to be clear, I think the tracking method added by the patch is
sufficient AFAICS for the problems we were able to discover. The
concern I was trying to express is that we seem to be duct-taping
holes in our earlier chosen design to track partitions separately from
the range table. If we had decided to add partitions to the range
table as "extra" target relations from the get-go, both the issues I
mentioned with cached plans -- partitions not being counted as a
dependency and partitions not being locked before execution -- would
have been prevented. I haven't fully grasped how invasive that design
would be, but it sure sounds like it would be a bit more robust.
Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...
(the alternative of adding partitions to the range table needs further
investigation)
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v20-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v20-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From c7e06565762833fd0615be5132d6a00ae0c579c1 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Mon, 1 Mar 2021 11:00:14 +1100
Subject: [PATCH v20 1/5] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or predicate,
column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 22 ++
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 29 +-
src/backend/optimizer/plan/setrefs.c | 20 +
src/backend/optimizer/util/clauses.c | 525 ++++++++++++++++++++++++++-
src/backend/utils/cache/plancache.c | 31 ++
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 4 +-
13 files changed, 641 insertions(+), 16 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df6b8..e1d16fc1cf 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,27 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec2c4..da91cbd2b1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432bfe1..6493a03ff8 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58e86..c5e136e9c3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56bcaf..82eb1865bd 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -247,7 +247,6 @@ static bool group_by_has_partkey(RelOptInfo *input_rel,
List *groupClause);
static int common_prefix_cmp(const void *a, const void *b);
-
/*****************************************************************************
*
* Query optimizer entry point
@@ -305,6 +304,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +316,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +334,15 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,9 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+ /* For AcquireExecutorLocks(). */
+ if (IsModifySupportedInParallelMode(parse->commandType))
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 42f088ad71..6a1c65e7ac 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,7 +15,10 @@
*/
#include "postgres.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +27,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "tcop/utility.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@@ -260,6 +265,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If modifying a partitioned table, add its parallel-safety-checked
+ * partitions too to glob->relationOids, to register them as plan
+ * dependencies. This is only really needed in the case of a parallel
+ * plan, so that if parallel-unsafe properties are subsequently defined
+ * on the partitions, the cached parallel plan will be invalidated and
+ * a non-parallel plan will be generated.
+ */
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+ }
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..d1ec5a8e9c 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,15 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +170,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +566,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +609,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -687,6 +716,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with
+ * a modifying CTE does not have that flag set, due to a bug in the
+ * query rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -757,6 +807,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +840,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +851,460 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *)ii_Predicate, context))
+ {
+ found_max_hazard = true;
+ }
+ }
+ }
+
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel safety.
+ * DEFAULT values of DOMAIN-type columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan of the
+ * query tree in standard_planner().
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (rel->rd_rel->relkind == RELKIND_PARTITIONED_TABLE)
+ {
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ {
+ return true;
+ }
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ {
+ return true;
+ }
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ command_type,
+ context);
+ table_close(part_rel, AccessShareLock);
+
+ /* Record the partition as a potential plan dependency. */
+ glob->partitionOids =
+ lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ {
+ return true;
+ }
+
+ /*
+ * If any triggers exist, check that they are parallel safe.
+ */
+ if (rel->trigdesc != NULL)
+ {
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ {
+ return true;
+ }
+ }
+
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only parallel INSERT is currently supported,
+ * only command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
+ {
+ /*
+ * Column default expressions for columns in the target-list are already
+ * being checked for parallel-safety in the max_parallel_hazard() scan
+ * of the query tree in standard_planner().
+ * Note that even though column defaults may be specified separately for
+ * each partition in a partitioned table, a partition's default value is
+ * not applied when inserting a tuple through a partitioned table.
+ */
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ {
+ return true;
+ }
+ }
+ }
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ return true;
+ }
+ }
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_possible_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_possible_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ Assert(IsModifySupportedInParallelMode(parse->commandType));
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a0950489d..75d1088901 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,15 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL &&
+ plannedstmt->parallelModeNeeded)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0fc9f..86405a274e 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104d0b..95292d7573 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..e2f0fe3aa6 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,7 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern char max_parallel_hazard_for_modify(Query *parse, char initial_max_parallel_hazard);
+extern bool is_parallel_possible_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v20-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchapplication/octet-stream; name=v20-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc.patchDownload
From 7d65f95f78f678f6351e2424aa80bcb99462cc94 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 17:28:59 +1100
Subject: [PATCH v20 2/5] Parallel SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 1292 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 656 +++++++++
5 files changed, 1953 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..9705de7622
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,1292 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+insert into names9 select * from names;
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7342..e280198b17 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97324..6a57e889a1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..f881f3d791
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,656 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_data2(like test_data);
+insert into test_data2 select i from generate_series(1,10000) i;
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with before+after parallel-safe stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-unsafe stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger insert_before_trigger_unsafe before insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_before_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
+drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v20-0003-Add-new-parallel-dml-GUC-and-table-options.patchapplication/octet-stream; name=v20-0003-Add-new-parallel-dml-GUC-and-table-options.patchDownload
From 3a1c8e03872db44c192744388dd1b89b33402d2b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 23:51:14 +1100
Subject: [PATCH v20 3/5] Add new GUC option "enable_parallel_dml" (boolean)
and table option "parallel_dml_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_dml" is
added, to allow parallel DML to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel dml with finer granularity, to enable/disable the use of parallel
dml for specific tables.
The new table option "parallel_dml_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++
src/backend/access/common/reloptions.c | 25 ++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 ++++++++--
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b5718fc136..3fba832711 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5061,6 +5061,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-dml" xreflabel="enable_parallel_dml">
+ <term><varname>enable_parallel_dml</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_dml</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ table-modification commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for table-modification. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ table-modification.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..ecb047021d 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_dml_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..5d8f842ddd 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-dml-enabled" xreflabel="parallel_dml_enabled">
+ <term><literal>parallel_dml_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_dml_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel DML for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-dml"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for table-modification. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel DML for this table.
+ Note that if the target table of the parallel DML is partitioned, the
+ <literal>parallel_dml_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..938131a29a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_dml_enabled",
+ "Enables \"parallel dml\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_dml_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_dml_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..c04b262f26 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_dml = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index d1ec5a8e9c..0fbfa7f5a6 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1257,8 +1257,10 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
* table-modification statement.
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_dml is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_dml_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1269,11 +1271,16 @@ bool
is_parallel_possible_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
Assert(IsModifySupportedInParallelMode(parse->commandType));
+ if (!enable_parallel_dml)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1303,7 +1310,23 @@ is_parallel_possible_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelDML(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d626731723..27b0c0c099 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2048,6 +2048,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_dml", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for table-modification commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_dml,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..5c1266ef39 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_dml = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..e7e4af795e 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_dml_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..cd1f0d450d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_dml;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..7dce70ecea 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelDML
+ * Returns the relation's parallel_dml_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelDML(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_dml_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_dml_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 9705de7622..62e86cc214 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -85,12 +85,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -914,9 +944,28 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..6c25bc1e2f 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
+ enable_parallel_dml | off
enable_parallel_hash | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index f881f3d791..55ba2e2b5b 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -105,13 +105,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
@@ -394,10 +415,25 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.27.0
v20-0004-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v20-0004-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 2c4e3e61633b566443234ceff663b795c23a030b Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Wed, 10 Feb 2021 14:50:40 +1100
Subject: [PATCH v20 4/5] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 ++++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 +++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 264 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 28 ++-
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 511 insertions(+), 163 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 3b435c107d..58730b7dbd 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2294,10 +2297,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index e1d16fc1cf..d6462d236c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a
* FullTransactionId, to be included in the transaction state that is
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c04b262f26..a1a10972cb 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 906cab7053..2a3ad42f90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -345,7 +345,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 82eb1865bd..85659acd75 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -322,10 +325,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1797,7 +1801,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1845,6 +1962,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2285,96 +2403,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2393,6 +2448,13 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why hasn't this been
+ * set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7562,7 +7624,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 6a1c65e7ac..8f859484df 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -257,6 +257,7 @@ set_plan_references(PlannerInfo *root, Plan *plan)
PlannerGlobal *glob = root->glob;
int rtoffset = list_length(glob->finalrtable);
ListCell *lc;
+ Plan *finalPlan;
/*
* Add all the query's RTEs to the flattened rangetable. The live ones
@@ -322,7 +323,9 @@ set_plan_references(PlannerInfo *root, Plan *plan)
}
/* Now fix the Plan tree */
- return set_plan_refs(root, plan, rtoffset);
+ finalPlan = set_plan_refs(root, plan, rtoffset);
+
+ return finalPlan;
}
/*
@@ -1086,6 +1089,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 69b83071cf..e6af7d3067 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3553,6 +3553,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3563,10 +3564,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3583,47 +3584,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3639,6 +3615,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e31ad6204e..27272ec2c8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index cd1f0d450d..ef651337f2 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 54f4b782fc..cb36413e5b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -267,7 +267,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 7dce70ecea..1197ae24d3 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -598,15 +599,15 @@ typedef struct PartitionedOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v20-0005-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v20-0005-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From f69d848f43be51bbc52ee96325c760d5e44662de Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 25 Feb 2021 23:39:40 +1100
Subject: [PATCH v20 5/5] Parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++-
src/test/regress/expected/insert_parallel.out | 142 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 +--
3 files changed, 158 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 62e86cc214..868282437e 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -122,14 +122,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -165,7 +165,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -212,15 +212,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -305,9 +305,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -344,9 +344,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -386,9 +386,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -436,9 +436,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -471,9 +471,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -507,9 +507,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -545,9 +545,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -585,9 +585,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -659,21 +659,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -702,7 +702,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -738,9 +738,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -801,14 +801,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -857,7 +857,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -886,14 +886,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -969,9 +969,9 @@ alter table parttable1 set (parallel_dml_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1033,9 +1033,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1072,7 +1072,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -1093,11 +1093,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1138,7 +1138,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1195,13 +1195,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1228,9 +1228,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 55ba2e2b5b..643de1b129 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -136,7 +136,7 @@ alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -148,7 +148,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -161,7 +161,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -305,14 +305,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -320,7 +320,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -373,7 +373,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -389,7 +389,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -397,7 +397,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -496,7 +496,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with before+after parallel-safe stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -546,7 +546,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with before+after parallel-restricted stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -592,7 +592,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
On Mon, Mar 1, 2021 at 12:38 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 26, 2021 at 5:50 PM Amit Langote <amitlangote09@gmail.com> wrote:
On Fri, Feb 26, 2021 at 3:35 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Feb 26, 2021 at 4:07 PM Amit Langote <amitlangote09@gmail.com> wrote:
The attached patch fixes this, although I am starting to have second
thoughts about how we're tracking partitions in this patch. Wondering
if we should bite the bullet and add partitions into the main range
table instead of tracking them separately in partitionOids, which
might result in a cleaner patch overall.Thanks Amit,
I was able to reproduce the problem using your instructions (though I
found I had to run that explain an extra time, in order to hit the
breakpoint).
Also, I can confirm that the problem doesn't occur after application
of your patch.
I'll leave it to your better judgement as to what to do next - if you
feel the current tracking method is not sufficientJust to be clear, I think the tracking method added by the patch is
sufficient AFAICS for the problems we were able to discover. The
concern I was trying to express is that we seem to be duct-taping
holes in our earlier chosen design to track partitions separately from
the range table. If we had decided to add partitions to the range
table as "extra" target relations from the get-go, both the issues I
mentioned with cached plans -- partitions not being counted as a
dependency and partitions not being locked before execution -- would
have been prevented. I haven't fully grasped how invasive that design
would be, but it sure sounds like it would be a bit more robust.Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...
Thanks Greg.
(the alternative of adding partitions to the range table needs further
investigation)
I looked at this today with an intention to write and post a PoC
patch, but was quickly faced with the task of integrating that design
with how ModifyTable works for inserts. Specifically if, in addition
to adding partitions to the range table, we also their RT indexes to
ModifyTable.resultRelations, then maybe we'll need to rethink our
executor tuple routing code. That code currently tracks the insert's
target partitions separately from the range table, exactly because
they are not there to begin with. So if we change the latter as in
this hypothetical PoC, maybe we should also revisit the former. Also,
it would not be great that the planner's arrays in PlannerInfo would
get longer as a result of Query.rtable getting longer by adding
partitions, thus making all the loops over those arrays slower for no
benefit.
So, let's do this in a follow-on patch, if at all, instead of
considering this topic a blocker for committing the current set of
patches.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Mon, Mar 1, 2021 at 9:08 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...
Few comments:
==============
1.
"Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use."
This point in the commit message seems a bit misleading to me because
IIUC we need to use transaction id only in the master backend for the
purpose of this patch. And, we are doing this at an early stage
because we don't allow to allocate it in parallel-mode. I think here
we should mention in some way that this has a disadvantage that if the
underneath select doesn't return any row then this transaction id
allocation would not be of use. However, that shouldn't happen in
practice in many cases. But, if I am missing something and this is
required by parallel workers then we can ignore what I said.
2.
/*
+ * Prepare for entering parallel mode by assigning a
+ * FullTransactionId, to be included in the transaction state that is
+ * serialized in the parallel DSM.
+ */
+ (void) GetCurrentTransactionId();
+ }
Similar to the previous comment this also seems to indicate that we
require TransactionId for workers. If that is not the case then this
comment also needs to be modified.
3.
static int common_prefix_cmp(const void *a, const void *b);
-
/*****************************************************************************
Spurious line removal.
4.
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data using a CTE, or if this is a cursor operation, or if
+ * GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
This comment change is not required because this is quite similar to
what we do for CTAS. Your further comment changes in this context are
sufficient.
5.
+ (IsModifySupportedInParallelMode(parse->commandType) &&
+ is_parallel_possible_for_modify(parse))) &&
I think it would be better if we move the check
IsModifySupportedInParallelMode inside
is_parallel_possible_for_modify. Also, it might be better to name this
function as is_parallel_allowed_for_modify.
6.
@@ -260,6 +265,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If modifying a partitioned table, add its parallel-safety-checked
+ * partitions too to glob->relationOids, to register them as plan
+ * dependencies. This is only really needed in the case of a parallel
+ * plan, so that if parallel-unsafe properties are subsequently defined
+ * on the partitions, the cached parallel plan will be invalidated and
+ * a non-parallel plan will be generated.
+ */
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+ }
+
Isn't it possible to add these partitionOids in set_plan_refs with the
T_Gather(Merge) node handling? That looks like a more natural place to
me, if that is feasible then we don't need parallelModeNeeded check
and maybe we don't need to even check IsModifySupportedInParallelMode
but I am less sure of the second check requirement.
7.
+#include "access/table.h"
+#include "access/xact.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +27,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
I think apart from xact.h, we don't need new additional includes.
8. I see that in function target_rel_max_parallel_hazard, we don't
release the lock on the target table after checking parallel-safety
but then in function target_rel_max_parallel_hazard_recurse, we do
release the lock on partition tables after checking their
parallel-safety. Can we add some comments to explain these two cases?
9. I noticed that the tests added for the first patch in
v18-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc take
even more time than select_parallel. I think we should see if we can
reduce the timing of this test. I haven't studied it in detail but
maybe some Inserts execution can be avoided. In some cases like below
just checking the plan should be sufficient. I think you can try to
investigate and see how much we can reduce it without affecting on
code-coverage of newly added code.
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
--
With Regards,
Amit Kapila.
On Tue, Mar 2, 2021 at 11:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 1, 2021 at 9:08 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...Few comments:
==============
1.
"Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state which
is then serialized in the parallel DSM for the parallel workers to use."This point in the commit message seems a bit misleading to me because
IIUC we need to use transaction id only in the master backend for the
purpose of this patch. And, we are doing this at an early stage
because we don't allow to allocate it in parallel-mode. I think here
we should mention in some way that this has a disadvantage that if the
underneath select doesn't return any row then this transaction id
allocation would not be of use. However, that shouldn't happen in
practice in many cases. But, if I am missing something and this is
required by parallel workers then we can ignore what I said.
I'll remove the part that says "which is then serialized ...", as this
may imply that the patch is doing this (which of course it is not,
Postgres always serializes the transaction state in the parallel DSM
for the parallel workers to use).
The acquiring of the TransactionId up-front, prior to entering
parallel-mode, is absolutely required because heap_insert() lazily
tries to get the current transaction-id and attempts to assign one if
the current transaction state hasn't got one, and this assignment is
not allowed in parallel-mode ("ERROR: cannot assign XIDs during a
parallel operation"), and this mode is required for use of parallel
SELECT.
I'll add the part about the disadvantage of the transaction-id not
being used if the underlying SELECT doesn't return any rows.
2. /* + * Prepare for entering parallel mode by assigning a + * FullTransactionId, to be included in the transaction state that is + * serialized in the parallel DSM. + */ + (void) GetCurrentTransactionId(); + }Similar to the previous comment this also seems to indicate that we
require TransactionId for workers. If that is not the case then this
comment also needs to be modified.
I'll update to comment to remove the part about the serialization (as
this always happens, not a function of the patch) and mention it is
needed to avoid attempting to assign a FullTransactionId in
parallel-mode.
3.
static int common_prefix_cmp(const void *a, const void *b);-
/*****************************************************************************Spurious line removal.
I will reinstate that empty line.
4. * Assess whether it's feasible to use parallel mode for this query. We * can't do this in a standalone backend, or if the command will try to - * modify any data, or if this is a cursor operation, or if GUCs are set - * to values that don't permit parallelism, or if parallel-unsafe - * functions are present in the query tree. + * modify any data using a CTE, or if this is a cursor operation, or if + * GUCs are set to values that don't permit parallelism, or if + * parallel-unsafe functions are present in the query tree.This comment change is not required because this is quite similar to
what we do for CTAS. Your further comment changes in this context are
sufficient.
An INSERT modifies data, so according to the original comment, then
it's not feasible to use parallel mode, because the command tries to
modify data ("We can't do this in a standalone backend, or if the
command will try to modify any data ...").
Except now we need to use parallel-mode for "INSERT with parallel
SELECT", and INSERT is a command that modifies data.
So isn't the comment change actually needed?
5. + (IsModifySupportedInParallelMode(parse->commandType) && + is_parallel_possible_for_modify(parse))) &&I think it would be better if we move the check
IsModifySupportedInParallelMode inside
is_parallel_possible_for_modify.
The reason it is done outside of is_parallel_possible_for_modify() is
to avoid the function overhead of calling
is_parallel_possible_for_modify() for SELECT statements, only to
return without doing anything. Note also that
IsModifySupportedInParallelMode() is an inline function.
Also, it might be better to name this
function as is_parallel_allowed_for_modify.
I do tend to think that in this case "possible" is better than "allowed".
Only the "parallel_dml" GUC test is checking for something that is "allowed".
The other two checks are checking for things that determine whether
parallel-mode is even "possible".
6.
@@ -260,6 +265,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);+ /* + * If modifying a partitioned table, add its parallel-safety-checked + * partitions too to glob->relationOids, to register them as plan + * dependencies. This is only really needed in the case of a parallel + * plan, so that if parallel-unsafe properties are subsequently defined + * on the partitions, the cached parallel plan will be invalidated and + * a non-parallel plan will be generated. + */ + if (IsModifySupportedInParallelMode(root->parse->commandType)) + { + if (glob->partitionOids != NIL && glob->parallelModeNeeded) + glob->relationOids = + list_concat(glob->relationOids, glob->partitionOids); + } +Isn't it possible to add these partitionOids in set_plan_refs with the
T_Gather(Merge) node handling? That looks like a more natural place to
me, if that is feasible then we don't need parallelModeNeeded check
and maybe we don't need to even check IsModifySupportedInParallelMode
but I am less sure of the second check requirement.
There may be multiple Gather/GatherMerge nodes in the plan (when they
are not top-level nodes), and I think by moving this code into
set_plan_refs() you risk adding those partitionOids multiple times to
glob->relationOids, when the Gather/GatherMerge nodes are traversed
(note that set_plan_refs() is called from set_plan_references() and is
recursive).
Leaving the code where it is is set_plan_references() guarantees that
the partitionOids can only be added ONCE.
7.
+#include "access/table.h"
+#include "access/xact.h"
#include "access/transam.h"
+#include "catalog/pg_class.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
@@ -24,6 +27,8 @@
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/tlist.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"I think apart from xact.h, we don't need new additional includes.
OK, I'll remove, it seems those other headers are getting dragged in
from the existing headers.
8. I see that in function target_rel_max_parallel_hazard, we don't
release the lock on the target table after checking parallel-safety
but then in function target_rel_max_parallel_hazard_recurse, we do
release the lock on partition tables after checking their
parallel-safety. Can we add some comments to explain these two cases?
It looks like the comment on the first case was lost when the code was
integrated into max_parallel_hazard().
The target relation is always locked during the parse/analyze phase
and left locked until end-of-transaction. So the lock here is just
incrementing a reference count (AFAIK). Note that originally NoLock
was passed to table_open(), which would have a similar overall effect,
as the table has already been locked and will be unlocked at
end-of-transaction.
I'll add comments in both cases.
9. I noticed that the tests added for the first patch in
v18-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-tests-and-doc take
even more time than select_parallel. I think we should see if we can
reduce the timing of this test. I haven't studied it in detail but
maybe some Inserts execution can be avoided. In some cases like below
just checking the plan should be sufficient. I think you can try to
investigate and see how much we can reduce it without affecting on
code-coverage of newly added code.+-- +-- Parallel unsafe column default, should not use a parallel plan +-- +explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data; + QUERY PLAN +----------------------------- + Insert on testdef + -> Seq Scan on test_data +(2 rows) + +insert into testdef(a,c,d) select a,a*4,a*8 from test_data; +select * from testdef order by a; + a | b | c | d +----+---+----+---- + 1 | 5 | 4 | 8 + 2 | 5 | 8 | 16 + 3 | 5 | 12 | 24 + 4 | 5 | 16 | 32 + 5 | 5 | 20 | 40 + 6 | 5 | 24 | 48 + 7 | 5 | 28 | 56 + 8 | 5 | 32 | 64 + 9 | 5 | 36 | 72 + 10 | 5 | 40 | 80 +(10 rows) +
OK, I'll try to remove INSERT executions (and checking) for the cases
primarily checking the type of plan being generated, where the related
INSERT functionality has been previously tested.
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Mar 3, 2021 at 12:52 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Tue, Mar 2, 2021 at 11:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 1, 2021 at 9:08 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
2. /* + * Prepare for entering parallel mode by assigning a + * FullTransactionId, to be included in the transaction state that is + * serialized in the parallel DSM. + */ + (void) GetCurrentTransactionId(); + }Similar to the previous comment this also seems to indicate that we
require TransactionId for workers. If that is not the case then this
comment also needs to be modified.I'll update to comment to remove the part about the serialization (as
this always happens, not a function of the patch) and mention it is
needed to avoid attempting to assign a FullTransactionId in
parallel-mode.
Okay, but just use TransactionId in comments if there is a need, we
still don't use FullTransactionId for the heap.
4. * Assess whether it's feasible to use parallel mode for this query. We * can't do this in a standalone backend, or if the command will try to - * modify any data, or if this is a cursor operation, or if GUCs are set - * to values that don't permit parallelism, or if parallel-unsafe - * functions are present in the query tree. + * modify any data using a CTE, or if this is a cursor operation, or if + * GUCs are set to values that don't permit parallelism, or if + * parallel-unsafe functions are present in the query tree.This comment change is not required because this is quite similar to
what we do for CTAS. Your further comment changes in this context are
sufficient.An INSERT modifies data, so according to the original comment, then
it's not feasible to use parallel mode, because the command tries to
modify data ("We can't do this in a standalone backend, or if the
command will try to modify any data ...").
Except now we need to use parallel-mode for "INSERT with parallel
SELECT", and INSERT is a command that modifies data.
So isn't the comment change actually needed?
If we want to change, we can say "if the command will try to modify
any data except for inserts ..." or something like that but saying
only about CTE is not correct because then what about updates and
deletes.
5. + (IsModifySupportedInParallelMode(parse->commandType) && + is_parallel_possible_for_modify(parse))) &&I think it would be better if we move the check
IsModifySupportedInParallelMode inside
is_parallel_possible_for_modify.The reason it is done outside of is_parallel_possible_for_modify() is
to avoid the function overhead of calling
is_parallel_possible_for_modify() for SELECT statements, only to
return without doing anything. Note also that
IsModifySupportedInParallelMode() is an inline function.
I don't see any reason to be worried about that here. It is more
important for code and checks to look simpler.
Also, it might be better to name this
function as is_parallel_allowed_for_modify.I do tend to think that in this case "possible" is better than "allowed".
Only the "parallel_dml" GUC test is checking for something that is "allowed".
The other two checks are checking for things that determine whether
parallel-mode is even "possible".
I think I don't like this proposed name for the function. See, if you
can think of something better.
6.
@@ -260,6 +265,21 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);+ /* + * If modifying a partitioned table, add its parallel-safety-checked + * partitions too to glob->relationOids, to register them as plan + * dependencies. This is only really needed in the case of a parallel + * plan, so that if parallel-unsafe properties are subsequently defined + * on the partitions, the cached parallel plan will be invalidated and + * a non-parallel plan will be generated. + */ + if (IsModifySupportedInParallelMode(root->parse->commandType)) + { + if (glob->partitionOids != NIL && glob->parallelModeNeeded) + glob->relationOids = + list_concat(glob->relationOids, glob->partitionOids); + } +Isn't it possible to add these partitionOids in set_plan_refs with the
T_Gather(Merge) node handling? That looks like a more natural place to
me, if that is feasible then we don't need parallelModeNeeded check
and maybe we don't need to even check IsModifySupportedInParallelMode
but I am less sure of the second check requirement.There may be multiple Gather/GatherMerge nodes in the plan (when they
are not top-level nodes), and I think by moving this code into
set_plan_refs() you risk adding those partitionOids multiple times to
glob->relationOids, when the Gather/GatherMerge nodes are traversed
(note that set_plan_refs() is called from set_plan_references() and is
recursive).
Leaving the code where it is is set_plan_references() guarantees that
the partitionOids can only be added ONCE.
Okay, is there a reason to use IsModifySupportedInParallelMode? Isn't
the second check sufficient?
--
With Regards,
Amit Kapila.
On Mon, Mar 1, 2021 at 9:08 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...
(the alternative of adding partitions to the range table needs further
investigation)
I was reviewing your latest patch and I have a few comments.
In patch 0001
1.
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
…….
+ /*
+ * Column default expressions and check constraints are only applicable to
+ * INSERT and UPDATE, but since only parallel INSERT is currently supported,
+ * only command_type==CMD_INSERT is checked here.
+ */
+ if (command_type == CMD_INSERT)
If we have an assert at the beginning of the function, then why do we
want to put the if check here?
2.
In patch 0004, We are still charging the parallel_tuple_cost for each
tuple, are we planning to do something about this? I mean after this
patch tuple will not be transferred through the tuple queue, so we
should not add that cost.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Mar 3, 2021 at 10:45 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Mar 1, 2021 at 9:08 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
Posting an updated set of patches that includes Amit Langote's patch
to the partition tracking scheme...
(the alternative of adding partitions to the range table needs further
investigation)I was reviewing your latest patch and I have a few comments.
In patch 0001 1. +static bool +target_rel_max_parallel_hazard_recurse(Relation rel, + CmdType command_type, + max_parallel_hazard_context *context) +{ + TupleDesc tupdesc; + int attnum; + + /* Currently only CMD_INSERT is supported */ + Assert(command_type == CMD_INSERT); ……. + /* + * Column default expressions and check constraints are only applicable to + * INSERT and UPDATE, but since only parallel INSERT is currently supported, + * only command_type==CMD_INSERT is checked here. + */ + if (command_type == CMD_INSERT)If we have an assert at the beginning of the function, then why do we
want to put the if check here?
Asserts are normally only enabled in a debug-build, so for a
release-build that Assert has no effect.
The Assert is being used as a sanity-check that the function is only
currently getting called for INSERT, because that's all it currently
supports.
Further code below specifically checks for INSERT because the block
contains code that is specific to INSERT (and actually DELETE too, but
that is not currently supported - code is commented accordingly).
In future, this function will support DELETE and UPDATE, and the
Assert serves to alert the developer ASAP that it needs updating to
support those.
2.
In patch 0004, We are still charging the parallel_tuple_cost for each
tuple, are we planning to do something about this? I mean after this
patch tuple will not be transferred through the tuple queue, so we
should not add that cost.
I believe that for Parallel INSERT, cost_modifytable() will set
path->path.rows to 0 (unless there is a RETURNING list), so, for
example, in cost_gather(), it will not add to the run_cost as
"run_cost += parallel_tuple_cost * path->path.rows;"
Regards,
Greg Nancarrow
Fujitsu Australia
On Wed, Mar 3, 2021 at 5:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Asserts are normally only enabled in a debug-build, so for a
release-build that Assert has no effect.
The Assert is being used as a sanity-check that the function is only
currently getting called for INSERT, because that's all it currently
supports.
I agree that assert is only for debug build, but once we add and
assert that means we are sure that it should only be called for insert
and if it is called for anything else then it is a programming error
from the caller's side. So after the assert, adding if check for the
same condition doesn't look like a good idea. That means we think
that the code can hit assert in the debug mode so we need an extra
protection in the release mode.
2.
In patch 0004, We are still charging the parallel_tuple_cost for each
tuple, are we planning to do something about this? I mean after this
patch tuple will not be transferred through the tuple queue, so we
should not add that cost.I believe that for Parallel INSERT, cost_modifytable() will set
path->path.rows to 0 (unless there is a RETURNING list), so, for
example, in cost_gather(), it will not add to the run_cost as
"run_cost += parallel_tuple_cost * path->path.rows;"
But the cost_modifytable is setting the number of rows to 0 in
ModifyTablePath whereas the cost_gather will multiply the rows from
the GatherPath. I can not see the rows from GatherPath is ever set to
0.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Thu, Mar 4, 2021 at 1:07 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Mar 3, 2021 at 5:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Asserts are normally only enabled in a debug-build, so for a
release-build that Assert has no effect.
The Assert is being used as a sanity-check that the function is only
currently getting called for INSERT, because that's all it currently
supports.I agree that assert is only for debug build, but once we add and
assert that means we are sure that it should only be called for insert
and if it is called for anything else then it is a programming error
from the caller's side. So after the assert, adding if check for the
same condition doesn't look like a good idea. That means we think
that the code can hit assert in the debug mode so we need an extra
protection in the release mode.
The if-check isn't there for "extra protection".
It's to help with future changes; inside that if-block is code only
applicable to INSERT (and to UPDATE - sorry, before I said DELETE), as
the code-comment indicates, whereas the rest of the function is
generic to all command types. I don't see any problem with having this
if-block here, to help in this way, when other command types are
added.
2.
In patch 0004, We are still charging the parallel_tuple_cost for each
tuple, are we planning to do something about this? I mean after this
patch tuple will not be transferred through the tuple queue, so we
should not add that cost.I believe that for Parallel INSERT, cost_modifytable() will set
path->path.rows to 0 (unless there is a RETURNING list), so, for
example, in cost_gather(), it will not add to the run_cost as
"run_cost += parallel_tuple_cost * path->path.rows;"But the cost_modifytable is setting the number of rows to 0 in
ModifyTablePath whereas the cost_gather will multiply the rows from
the GatherPath. I can not see the rows from GatherPath is ever set to
0.
OK, I see the problem now.
It works the way I described, but currently there's a problem with
where it's getting the rows for the GatherPath, so there's a
disconnect.
When generating the GatherPaths, it's currently always taking the
rel's (possibly estimated) row-count, rather than using the rows from
the cheapest_partial_path (the subpath: ModifyTablePath). See
generate_gather_paths().
So when generate_useful_gather_paths() is called from the planner, for
the added partial paths for Parallel INSERT, it should be passing
"true" for the "override_rows" parameter, not "false".
So I think that in the 0004 patch, the if-block:
+ if (parallel_modify_partial_path_added)
+ {
+ final_rel->rows = current_rel->rows; /* ??? why
hasn't this been
+
* set above somewhere ???? */
+ generate_useful_gather_paths(root, final_rel, false);
+ }
+
can be reduced to:
+ if (parallel_modify_partial_path_added)
+ generate_useful_gather_paths(root, final_rel, true);
+
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Mar 4, 2021 at 7:16 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 1:07 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Mar 3, 2021 at 5:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
Asserts are normally only enabled in a debug-build, so for a
release-build that Assert has no effect.
The Assert is being used as a sanity-check that the function is only
currently getting called for INSERT, because that's all it currently
supports.I agree that assert is only for debug build, but once we add and
assert that means we are sure that it should only be called for insert
and if it is called for anything else then it is a programming error
from the caller's side. So after the assert, adding if check for the
same condition doesn't look like a good idea. That means we think
that the code can hit assert in the debug mode so we need an extra
protection in the release mode.The if-check isn't there for "extra protection".
It's to help with future changes; inside that if-block is code only
applicable to INSERT (and to UPDATE - sorry, before I said DELETE), as
the code-comment indicates, whereas the rest of the function is
generic to all command types. I don't see any problem with having this
if-block here, to help in this way, when other command types are
added.
I think for Update/Delete, we might not do parallel-safety checks by
calling target_rel_max_parallel_hazard_recurse especially because
partitions are handled differently for Updates and Deletes (see
inheritance_planner()). I think what Dilip is telling doesn't sound
unreasonable to me. So, even, if we want to extend it later by making
some checks specific to Inserts/Updates, we can do it at that time.
The comments you have at that place are sufficient to tell that in the
future we can use those checks for Updates as well. They will need
some adjustment if we remove that check but the intent is clear.
--
With Regards,
Amit Kapila.
On Thu, Mar 4, 2021 at 7:16 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 1:07 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Mar 3, 2021 at 5:50 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I agree that assert is only for debug build, but once we add and
assert that means we are sure that it should only be called for insert
and if it is called for anything else then it is a programming error
from the caller's side. So after the assert, adding if check for the
same condition doesn't look like a good idea. That means we think
that the code can hit assert in the debug mode so we need an extra
protection in the release mode.The if-check isn't there for "extra protection".
It's to help with future changes; inside that if-block is code only
applicable to INSERT (and to UPDATE - sorry, before I said DELETE), as
the code-comment indicates, whereas the rest of the function is
generic to all command types. I don't see any problem with having this
if-block here, to help in this way, when other command types are
added.
If that is the case then this check should also be added along with
that future patches, I mean when we will allow UPDATE then it makes
sense of that check and that time will have to get rid of that assert
as well. I mean complete function will be in sync. But now this
check looks a bit odd. I think that is my opinion but otherwise, I
don't have any strong objection to that check.
2.
But the cost_modifytable is setting the number of rows to 0 in
ModifyTablePath whereas the cost_gather will multiply the rows from
the GatherPath. I can not see the rows from GatherPath is ever set to
0.OK, I see the problem now.
It works the way I described, but currently there's a problem with
where it's getting the rows for the GatherPath, so there's a
disconnect.
When generating the GatherPaths, it's currently always taking the
rel's (possibly estimated) row-count, rather than using the rows from
the cheapest_partial_path (the subpath: ModifyTablePath). See
generate_gather_paths().
So when generate_useful_gather_paths() is called from the planner, for
the added partial paths for Parallel INSERT, it should be passing
"true" for the "override_rows" parameter, not "false".So I think that in the 0004 patch, the if-block:
+ if (parallel_modify_partial_path_added) + { + final_rel->rows = current_rel->rows; /* ??? why hasn't this been + * set above somewhere ???? */ + generate_useful_gather_paths(root, final_rel, false); + } +can be reduced to:
+ if (parallel_modify_partial_path_added) + generate_useful_gather_paths(root, final_rel, true); +
Okay. I will check this part after you provide an updated version. Thanks.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Thu, Mar 4, 2021 at 9:03 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think for Update/Delete, we might not do parallel-safety checks by
calling target_rel_max_parallel_hazard_recurse especially because
partitions are handled differently for Updates and Deletes (see
inheritance_planner()). I think what Dilip is telling doesn't sound
unreasonable to me. So, even, if we want to extend it later by making
some checks specific to Inserts/Updates, we can do it at that time.
The comments you have at that place are sufficient to tell that in the
future we can use those checks for Updates as well. They will need
some adjustment if we remove that check but the intent is clear.
+1
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Fri, Feb 26, 2021 at 10:37 AM Amit Langote <amitlangote09@gmail.com> wrote:
I realized that there is a race condition that will allow a concurrent
backend to invalidate a parallel plan (for insert into a partitioned
table) at a point in time when it's too late for plancache.c to detect
it. It has to do with how plancache.c locks the relations affected by
a cached query and its cached plan. Specifically,
AcquireExecutorLocks(), which locks the relations that need to be
locked before the plan could be considered safe to execute, does not
notice the partitions that would have been checked for parallel safety
when the plan was made. Given that AcquireExecutorLocks() only loops
over PlannedStmt.rtable and locks exactly the relations found there,
partitions are not locked. This means that a concurrent backend can,
for example, add an unsafe trigger to a partition before a parallel
worker inserts into it only to fail when it does.Steps to reproduce (tried with v19 set of patches):
drop table if exists rp, foo;
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
set plan_cache_mode to force_generic_plan;
set enable_parallel_dml to on;
deallocate all;
prepare q as insert into rp select * from foo where a%2 = 0;
explain analyze execute q;At this point, attach a debugger to this backend and set a breakpoint
in AcquireExecutorLocks() and execute q again:-- will execute the cached plan
explain analyze execute q;Breakpoint will be hit. Continue till return from the function and
stop. Start another backend and execute this:-- will go through, because no lock still taken on the partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();Back to the debugger, hit continue to let the plan execute. You
should see this error:ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement parallel workerThe attached patch fixes this,
One thing I am a bit worried about this fix is that after this for
parallel-mode, we will maintain partitionOids at two places, one in
plannedstmt->relationOids and the other in plannedstmt->partitionOids.
I guess you have to do that because, in AcquireExecutorLocks, we can't
find which relationOids are corresponding to partitionOids, am I
right? If so, why not just maintain them in plannedstmt->partitionOids
and then make PlanCacheRelCallback consider it? Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.
Thoughts?
--
With Regards,
Amit Kapila.
On Thu, Mar 4, 2021 at 4:37 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 26, 2021 at 10:37 AM Amit Langote <amitlangote09@gmail.com> wrote:
I realized that there is a race condition that will allow a concurrent
backend to invalidate a parallel plan (for insert into a partitioned
table) at a point in time when it's too late for plancache.c to detect
it. It has to do with how plancache.c locks the relations affected by
a cached query and its cached plan. Specifically,
AcquireExecutorLocks(), which locks the relations that need to be
locked before the plan could be considered safe to execute, does not
notice the partitions that would have been checked for parallel safety
when the plan was made. Given that AcquireExecutorLocks() only loops
over PlannedStmt.rtable and locks exactly the relations found there,
partitions are not locked. This means that a concurrent backend can,
for example, add an unsafe trigger to a partition before a parallel
worker inserts into it only to fail when it does.Steps to reproduce (tried with v19 set of patches):
drop table if exists rp, foo;
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
set plan_cache_mode to force_generic_plan;
set enable_parallel_dml to on;
deallocate all;
prepare q as insert into rp select * from foo where a%2 = 0;
explain analyze execute q;At this point, attach a debugger to this backend and set a breakpoint
in AcquireExecutorLocks() and execute q again:-- will execute the cached plan
explain analyze execute q;Breakpoint will be hit. Continue till return from the function and
stop. Start another backend and execute this:-- will go through, because no lock still taken on the partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();Back to the debugger, hit continue to let the plan execute. You
should see this error:ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement parallel workerThe attached patch fixes this,
One thing I am a bit worried about this fix is that after this for
parallel-mode, we will maintain partitionOids at two places, one in
plannedstmt->relationOids and the other in plannedstmt->partitionOids.
I guess you have to do that because, in AcquireExecutorLocks, we can't
find which relationOids are corresponding to partitionOids, am I
right? If so, why not just maintain them in plannedstmt->partitionOids
and then make PlanCacheRelCallback consider it? Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.
One more point I was thinking about is whether we need to worry about
locking indexes during prepared query execution (similar to what we do
for AcquireExecutorLocks). I think we shouldn't be bothered to lock
those or even retain lock during parallel-safety checks because one
cannot change index expression or predicate. Is my understanding
correct or am I missing something and we should be worried about them
as well.
--
With Regards,
Amit Kapila.
On Thu, Mar 4, 2021 at 10:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 26, 2021 at 10:37 AM Amit Langote <amitlangote09@gmail.com> wrote:
I realized that there is a race condition that will allow a concurrent
backend to invalidate a parallel plan (for insert into a partitioned
table) at a point in time when it's too late for plancache.c to detect
it. It has to do with how plancache.c locks the relations affected by
a cached query and its cached plan. Specifically,
AcquireExecutorLocks(), which locks the relations that need to be
locked before the plan could be considered safe to execute, does not
notice the partitions that would have been checked for parallel safety
when the plan was made. Given that AcquireExecutorLocks() only loops
over PlannedStmt.rtable and locks exactly the relations found there,
partitions are not locked. This means that a concurrent backend can,
for example, add an unsafe trigger to a partition before a parallel
worker inserts into it only to fail when it does.Steps to reproduce (tried with v19 set of patches):
drop table if exists rp, foo;
create table rp (a int) partition by range (a);
create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select generate_series(1, 1000000);
set plan_cache_mode to force_generic_plan;
set enable_parallel_dml to on;
deallocate all;
prepare q as insert into rp select * from foo where a%2 = 0;
explain analyze execute q;At this point, attach a debugger to this backend and set a breakpoint
in AcquireExecutorLocks() and execute q again:-- will execute the cached plan
explain analyze execute q;Breakpoint will be hit. Continue till return from the function and
stop. Start another backend and execute this:-- will go through, because no lock still taken on the partition
create or replace function make_table () returns trigger language
plpgsql as $$ begin create table bar(); return null; end; $$ parallel
unsafe;
create trigger ai_rp2 after insert on rp2 for each row execute
function make_table();Back to the debugger, hit continue to let the plan execute. You
should see this error:ERROR: cannot start commands during a parallel operation
CONTEXT: SQL statement "create table bar()"
PL/pgSQL function make_table() line 1 at SQL statement parallel workerThe attached patch fixes this,
One thing I am a bit worried about this fix is that after this for
parallel-mode, we will maintain partitionOids at two places, one in
plannedstmt->relationOids and the other in plannedstmt->partitionOids.
I guess you have to do that because, in AcquireExecutorLocks, we can't
find which relationOids are corresponding to partitionOids, am I
right? If so, why not just maintain them in plannedstmt->partitionOids
and then make PlanCacheRelCallback consider it?
Maybe Amit Langote can kindly comment on this, as it would involve
updates to his prior partition-related fixes.
Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.
It is doing that in v20 patch (what makes you think it isn't?).
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Mar 4, 2021 at 5:24 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 10:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.It is doing that in v20 patch (what makes you think it isn't?).
The below code snippet:
+ /* For AcquireExecutorLocks(). */
+ if (IsModifySupportedInParallelMode(parse->commandType))
+ result->partitionOids = glob->partitionOids;
I understand that you have a check for the parallel mode in
AcquireExecutorLocks but why can't we have it before adding that to
planned statement
--
With Regards,
Amit Kapila.
On Thu, Mar 4, 2021 at 11:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Mar 4, 2021 at 5:24 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 10:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.It is doing that in v20 patch (what makes you think it isn't?).
The below code snippet: + /* For AcquireExecutorLocks(). */ + if (IsModifySupportedInParallelMode(parse->commandType)) + result->partitionOids = glob->partitionOids;I understand that you have a check for the parallel mode in
AcquireExecutorLocks but why can't we have it before adding that to
planned statement
OK, I think I'm on the same wavelength now (sorry, I didn't realise
you're talking about PlannedStmt).
What I believe you're suggesting is in the planner where
partitionOids are "added" to the returned PlannedStmt, they should
only be added if glob->parallelModeNeeded is true:.
i.e.
/* For AcquireExecutorLocks(). */
if (glob->partitionOids != NIL && glob->parallelModeNeeded)
result->partitionOids = glob->partitionOids;
(seems reasonable to me, as then it will match the condition for which
glob->partitionOids are added to glob->relationOids)
then in plancache.c the check on parallelModeNeeded can be removed:
/* Lock partitions ahead of modifying them in parallel mode. */
if (rti == resultRelation &&
plannedstmt->partitionOids != NIL)
AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
rte->rellockmode, acquire);
Let me know if this matches what you were thinking.
Regards,
Greg Nancarrow
Fujitsu Australia
On Fri, Mar 5, 2021 at 5:07 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 11:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Mar 4, 2021 at 5:24 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 10:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.It is doing that in v20 patch (what makes you think it isn't?).
The below code snippet: + /* For AcquireExecutorLocks(). */ + if (IsModifySupportedInParallelMode(parse->commandType)) + result->partitionOids = glob->partitionOids;I understand that you have a check for the parallel mode in
AcquireExecutorLocks but why can't we have it before adding that to
planned statementOK, I think I'm on the same wavelength now (sorry, I didn't realise
you're talking about PlannedStmt).What I believe you're suggesting is in the planner where
partitionOids are "added" to the returned PlannedStmt, they should
only be added if glob->parallelModeNeeded is true:.i.e.
/* For AcquireExecutorLocks(). */
if (glob->partitionOids != NIL && glob->parallelModeNeeded)
result->partitionOids = glob->partitionOids;(seems reasonable to me, as then it will match the condition for which
glob->partitionOids are added to glob->relationOids)then in plancache.c the check on parallelModeNeeded can be removed:
/* Lock partitions ahead of modifying them in parallel mode. */
if (rti == resultRelation &&
plannedstmt->partitionOids != NIL)
AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
rte->rellockmode, acquire);Let me know if this matches what you were thinking.
Yes, that is what I was thinking. But I have another question as well
regarding tracking these partitions at two places (once in
plannedstmt->relationOids and the second time in
plannedstmt->partitionOids). I think it is better if you can prepare a
patch with all the comments raised till now leaving this last question
aside.
--
With Regards,
Amit Kapila.
On Fri, Mar 5, 2021 at 12:46 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Mar 5, 2021 at 5:07 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 11:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Mar 4, 2021 at 5:24 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Thu, Mar 4, 2021 at 10:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Also, in
standard_planner, we should add these partitionOids only for
parallel-mode.It is doing that in v20 patch (what makes you think it isn't?).
The below code snippet: + /* For AcquireExecutorLocks(). */ + if (IsModifySupportedInParallelMode(parse->commandType)) + result->partitionOids = glob->partitionOids;I understand that you have a check for the parallel mode in
AcquireExecutorLocks but why can't we have it before adding that to
planned statementOK, I think I'm on the same wavelength now (sorry, I didn't realise
you're talking about PlannedStmt).What I believe you're suggesting is in the planner where
partitionOids are "added" to the returned PlannedStmt, they should
only be added if glob->parallelModeNeeded is true:.i.e.
/* For AcquireExecutorLocks(). */
if (glob->partitionOids != NIL && glob->parallelModeNeeded)
result->partitionOids = glob->partitionOids;(seems reasonable to me, as then it will match the condition for which
glob->partitionOids are added to glob->relationOids)then in plancache.c the check on parallelModeNeeded can be removed:
/* Lock partitions ahead of modifying them in parallel mode. */
if (rti == resultRelation &&
plannedstmt->partitionOids != NIL)
AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
rte->rellockmode, acquire);Let me know if this matches what you were thinking.
Yes, that is what I was thinking. But I have another question as well
regarding tracking these partitions at two places (once in
plannedstmt->relationOids and the second time in
plannedstmt->partitionOids). I think it is better if you can prepare a
patch with all the comments raised till now leaving this last question
aside.
Posting an updated set of patches, that I believe address all the
recent feedback...
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v21-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v21-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 79e6ae342973fb45381fdb08d850aeb773953692 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 5 Mar 2021 13:30:02 +1100
Subject: [PATCH v21 1/1] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or predicate,
column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state. This
is necessary to prevent the INSERT from attempting to assign the TransactionId
whilst in parallel-mode, which is not allowed. This approach has a disadvantage
in that if the underlying SELECT does not return any rows, then the
TransactionId is not used, however that shouldn't happen in practice in many
cases.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 26 ++
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 27 +-
src/backend/optimizer/plan/setrefs.c | 13 +
src/backend/optimizer/util/clauses.c | 554 ++++++++++++++++++++++++++-
src/backend/utils/cache/plancache.c | 30 ++
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 3 +-
13 files changed, 664 insertions(+), 15 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df6b8..422979dc2e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,31 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a TransactionId.
+ * Failure to do this now would result in heap_insert() subsequently
+ * attempting to assign a TransactionId whilst in parallel-mode,
+ * which is not allowed.
+ * This approach has a disadvantage in that if the underlying SELECT
+ * does not return any rows, then the TransactionId is not used,
+ * however that shouldn't happen in practice in many cases.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec2c4..da91cbd2b1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432bfe1..6493a03ff8 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58e86..c5e136e9c3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56bcaf..2476203b46 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +317,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data (except for Insert), or if this is a cursor operation,
+ * or if GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +335,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ is_parallel_allowed_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,9 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+ /* For AcquireExecutorLocks(). */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 42f088ad71..20e3d263a6 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,6 +15,7 @@
*/
#include "postgres.h"
+#include "access/xact.h"
#include "access/transam.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -260,6 +261,18 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If the Oids of parallel-safety-checked partitions have been recorded,
+ * add them too to glob->relationOids, to register them as plan
+ * dependencies. This is only really needed in the case of a parallel
+ * plan, so that if parallel-unsafe properties are subsequently defined
+ * on the partitions, the cached parallel plan will be invalidated and
+ * a non-parallel plan will be generated.
+ */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..c5d5aaf7e5 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,20 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid,
+ max_parallel_hazard_context *context);
+static bool target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_checks_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +175,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +571,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +614,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -687,6 +721,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with
+ * a modifying CTE does not have that flag set, due to a bug in the
+ * query rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -757,6 +812,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +845,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +856,484 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ if (trigdesc == NULL)
+ return false;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *)ii_Predicate, context))
+ found_max_hazard = true;
+ }
+ }
+
+ index_close(index_rel, NoLock);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel-safety.
+ * Any DEFAULT-value expressions of DOMAIN-type columns are already being
+ * checked for parallel-safety in the existing max_parallel_hazard() target-list
+ * processing, so there's no need to repeat those checks here.
+ *
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+/*
+ * target_rel_partitions_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any partitions of a
+ * of a specified relation.
+ */
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+
+ if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+ return false;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ return true;
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ return true;
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ context->command_type,
+ context);
+ table_close(part_rel, NoLock);
+
+ /* Record the partition as a potential plan dependency. */
+ glob->partitionOids =
+ lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_checks_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any CHECK expressions or
+ * CHECK constraints related to the specified relation.
+ */
+static bool
+target_rel_checks_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+ int attnum;
+
+ tupdesc = RelationGetDescr(rel);
+ for (attnum = 0; attnum < tupdesc->natts; attnum++)
+ {
+ Form_pg_attribute att = TupleDescAttr(tupdesc, attnum);
+
+ /* We don't need info for dropped or generated attributes */
+ if (att->attisdropped || att->attgenerated)
+ continue;
+
+ /*
+ * If the column is of a DOMAIN type, determine whether that
+ * domain has any CHECK expressions that are not parallel-mode
+ * safe.
+ */
+ if (get_typtype(att->atttypid) == TYPTYPE_DOMAIN)
+ {
+ if (target_rel_domain_max_parallel_hazard(att->atttypid, context))
+ return true;
+ }
+ }
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (target_rel_partitions_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If any triggers exist, check that they are parallel-safe.
+ */
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ return true;
+
+ /*
+ * Column default expressions are only applicable to INSERT and UPDATE.
+ * For columns in the target-list, these are already being checked for
+ * parallel-safety in the max_parallel_hazard() scan of the query tree
+ * in standard_planner(), so there's no need to do it here.
+ * Note that even though column defaults may be specified separately for
+ * each partition in a partitioned table, a partition's default value is
+ * not applied when inserting a tuple through a partitioned table.
+ */
+
+ /*
+ * CHECK constraints are only applicable to INSERT and UPDATE.
+ * If any CHECK constraints exist, determine if they are parallel-safe.
+ */
+ if (target_rel_checks_max_parallel_hazard(rel, context))
+ return true;
+
+ return false;
+}
+
+/*
+ * is_parallel_allowed_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_allowed_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ if (!IsModifySupportedInParallelMode(parse->commandType))
+ return false;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a0950489d..084aa85c6e 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,14 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0fc9f..86405a274e 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104d0b..95292d7573 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..8d85b02514 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern bool is_parallel_allowed_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v21-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-basic-tests-and-doc.patchapplication/octet-stream; name=v21-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-basic-tests-and-doc.patchDownload
From 643975b24c6f33a3b5c8341213cb5fd489473cf8 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 18:05:03 +1100
Subject: [PATCH v21 1/1] Parallel SELECT for "INSERT INTO ... SELECT ..." -
basic tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 635 ++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 337 ++++++++++
5 files changed, 977 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..e80d954061
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,635 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 4 | 8
+ 2 | 5 | 8 | 16
+ 3 | 5 | 12 | 24
+ 4 | 5 | 16 | 32
+ 5 | 5 | 20 | 40
+ 6 | 5 | 24 | 48
+ 7 | 5 | 28 | 56
+ 8 | 5 | 32 | 64
+ 9 | 5 | 36 | 72
+ 10 | 5 | 40 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+---+----+----
+ 1 | 5 | 10 | 8
+ 2 | 5 | 10 | 16
+ 3 | 5 | 10 | 24
+ 4 | 5 | 10 | 32
+ 5 | 5 | 10 | 40
+ 6 | 5 | 10 | 48
+ 7 | 5 | 10 | 56
+ 8 | 5 | 10 | 64
+ 9 | 5 | 10 | 72
+ 10 | 5 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7342..e280198b17 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97324..6a57e889a1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..bf6a67fe32
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,337 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+insert into names2 select * from names;
+select * from names2 order by fullname_parallel_unsafe(first_name, last_name);
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+insert into names4 select * from names;
+select * from names4 order by fullname_parallel_restricted(first_name, last_name);
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+insert into testdef(a,d) select a,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
+select count(*), sum(a) from table_check_b;
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
--
2.27.0
v21-0003-Add-new-parallel-dml-GUC-and-table-options.patchapplication/octet-stream; name=v21-0003-Add-new-parallel-dml-GUC-and-table-options.patchDownload
From 25c1b45f466ee43d0bafacce8a1c7d68124810da Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 21:14:22 +1100
Subject: [PATCH v21 1/1] Add new GUC option "enable_parallel_dml" (boolean)
and table option "parallel_dml_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_dml" is
added, to allow parallel DML to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel dml with finer granularity, to enable/disable the use of parallel
dml for specific tables.
The new table option "parallel_dml_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++
src/backend/access/common/reloptions.c | 25 ++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 ++++++++--
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73596..cc4e0c200e 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-dml" xreflabel="enable_parallel_dml">
+ <term><varname>enable_parallel_dml</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_dml</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ table-modification commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for table-modification. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ table-modification.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..ecb047021d 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_dml_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..5d8f842ddd 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-dml-enabled" xreflabel="parallel_dml_enabled">
+ <term><literal>parallel_dml_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_dml_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel DML for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-dml"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for table-modification. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel DML for this table.
+ Note that if the target table of the parallel DML is partitioned, the
+ <literal>parallel_dml_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..938131a29a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_dml_enabled",
+ "Enables \"parallel dml\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_dml_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_dml_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..c04b262f26 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_dml = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index e05adc5016..83c21d868b 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1285,8 +1285,10 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
* table-modification statement.
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_dml is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_dml_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1297,12 +1299,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_dml)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1332,7 +1339,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelDML(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 4bcf705a30..55b85f6f56 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_dml", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for table-modification commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_dml,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..5c1266ef39 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_dml = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..e7e4af795e 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_dml_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..cd1f0d450d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_dml;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..7dce70ecea 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelDML
+ * Returns the relation's parallel_dml_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelDML(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_dml_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_dml_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index e80d954061..1f8c57fab8 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -471,9 +501,28 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..6c25bc1e2f 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
+ enable_parallel_dml | off
enable_parallel_hash | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index bf6a67fe32..736ae2fc48 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
@@ -223,10 +244,25 @@ truncate testdef;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.27.0
v21-0004-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v21-0004-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From 8c68efb5d0bd60f0df86ed019b357d9223416241 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 21:40:58 +1100
Subject: [PATCH v21 1/1] Parallel SELECT for "INSERT INTO ... SELECT ..." -
advanced tests.
---
src/test/regress/expected/insert_parallel.out | 659 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 321 ++++++++-
2 files changed, 976 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 1f8c57fab8..a2cb3056c4 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -209,6 +231,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -247,6 +296,86 @@ select * from para_insert_f1 where unique1 >= 9990 order by unique1;
9999 | PUAAAA
(10 rows)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -271,6 +400,209 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -395,6 +727,52 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+insert into names9 select * from names;
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -415,6 +793,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -470,6 +882,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -545,6 +986,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -570,17 +1069,25 @@ select count(*), sum(a) from table_check_b;
(1 row)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -593,19 +1100,28 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -616,8 +1132,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -666,6 +1217,104 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -673,6 +1322,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -681,4 +1332,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 736ae2fc48..ea5a1d9525 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -141,6 +167,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -153,6 +187,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
select count(*), sum(unique1) from para_insert_f1;
select * from para_insert_f1 where unique1 >= 9990 order by unique1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -161,7 +218,72 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -202,6 +324,32 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -216,6 +364,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -233,6 +394,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -268,6 +437,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -284,39 +493,80 @@ insert into table_check_b(a,b,c) select unique1, stringu1, stringu2 from tenk1;
select count(*), sum(a) from table_check_b;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -350,6 +600,67 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+insert into dom_table_u select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_u;
+
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
rollback;
@@ -361,6 +672,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -370,4 +683,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v21-0005-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v21-0005-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 3d0e2c2725537f98f71094f39b919cf2f9d2a078 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 22:44:54 +1100
Subject: [PATCH v21 1/1] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 +++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 ++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 269 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 23 ++
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 512 insertions(+), 162 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 3b435c107d..58730b7dbd 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2294,10 +2297,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 422979dc2e..7a6f9caf8c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a TransactionId.
* Failure to do this now would result in heap_insert() subsequently
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c04b262f26..a1a10972cb 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 906cab7053..2a3ad42f90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -345,7 +345,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index ca1f20ceca..ec86f291d3 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -323,10 +326,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1797,7 +1801,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1845,6 +1962,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2285,96 +2403,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2393,6 +2448,18 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ /*
+ * Generate gather paths according to the added partial paths for the
+ * parallel table-modification command.
+ * Note that true is passed for the "override_rows" parameter, so that
+ * the rows from the cheapest partial path (ModifyTablePath) are used,
+ * not the rel's (possibly estimated) rows.
+ */
+ generate_useful_gather_paths(root, final_rel, true);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7562,7 +7629,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 20e3d263a6..a720227ecc 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -1079,6 +1079,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 69b83071cf..e6af7d3067 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3553,6 +3553,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3563,10 +3564,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3583,47 +3584,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3639,6 +3615,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e31ad6204e..27272ec2c8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index cd1f0d450d..ef651337f2 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 54f4b782fc..cb36413e5b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -267,7 +267,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 7dce70ecea..1197ae24d3 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -598,15 +599,15 @@ typedef struct PartitionedOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v21-0006-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v21-0006-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 997921a64258e1011ed4e3c3ee25afcbc05e4f88 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 23:29:45 +1100
Subject: [PATCH v21 1/1] INSERT and/or SELECT for "INSERT INTO ... SELECT
..." - tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++-
src/test/regress/expected/insert_parallel.out | 142 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 +--
3 files changed, 158 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index a2cb3056c4..f38b07489d 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -122,14 +122,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -165,7 +165,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -212,15 +212,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -305,9 +305,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -344,9 +344,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -384,9 +384,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -434,9 +434,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -469,9 +469,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -505,9 +505,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -543,9 +543,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -583,9 +583,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -657,21 +657,21 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -700,7 +700,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -736,9 +736,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -799,14 +799,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -855,7 +855,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -884,14 +884,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -967,9 +967,9 @@ alter table parttable1 set (parallel_dml_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1031,9 +1031,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -1070,7 +1070,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -1091,11 +1091,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1136,7 +1136,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1193,13 +1193,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1226,9 +1226,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index ea5a1d9525..23a2f1eed6 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -136,7 +136,7 @@ alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -148,7 +148,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -161,7 +161,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -303,14 +303,14 @@ select * from names4 order by fullname_parallel_restricted(first_name, last_name
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -318,7 +318,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -371,7 +371,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -387,7 +387,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -395,7 +395,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -494,7 +494,7 @@ select count(*), sum(a) from table_check_b;
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -544,7 +544,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -590,7 +590,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
On Fri, Mar 5, 2021 at 8:24 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
In patch v21-0003-Add-new-parallel-dml-GUC-and-table-options, we are
introducing GUC (enable_parallel_dml) and table option
(parallel_dml_enabled) for this feature. I am a bit worried about
using *_dml in the names because it is quite possible that for
parallel updates and parallel deletes we might not need any such GUC.
The reason we mainly need here is due to checking of parallel-safety
of partitioned tables and updates/deletes handle partitioned tables
differently than inserts so those might not be that costly. It is
possible that they are costly due to a different reason but not sure
mapping those to one GUC or table option is a good idea. Can we
consider using *_insert instead? I think GUC having _insert can be
probably used for a parallel copy (from) as well which I think will
have a similar overhead.
--
With Regards,
Amit Kapila.
On Fri, Mar 5, 2021 at 9:35 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Mar 5, 2021 at 8:24 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
In patch v21-0003-Add-new-parallel-dml-GUC-and-table-options, we are
introducing GUC (enable_parallel_dml) and table option
(parallel_dml_enabled) for this feature. I am a bit worried about
using *_dml in the names because it is quite possible that for
parallel updates and parallel deletes we might not need any such GUC.
The reason we mainly need here is due to checking of parallel-safety
of partitioned tables and updates/deletes handle partitioned tables
differently than inserts so those might not be that costly. It is
possible that they are costly due to a different reason but not sure
mapping those to one GUC or table option is a good idea. Can we
consider using *_insert instead? I think GUC having _insert can be
probably used for a parallel copy (from) as well which I think will
have a similar overhead.
I'll need to think about that one.
I may be wrong, but I would have thought at least updates would have
similar parallel-safety checking requirements to inserts and would
have similar potential cost issues.
For the time being at least, I am posting an updated set of patches,
as I found that the additional parallel-safety checks on DOMAIN check
constraints to be somewhat inefficient and could also be better
integrated into max_parallel_hazard(). I also updated the basic tests
with a test case for this.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v22-0005-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v22-0005-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From c43acc28080c5dcd7105f20be80f6c4f3e599b9a Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 22:44:54 +1100
Subject: [PATCH v22 5/6] Enable parallel INSERT and/or SELECT for "INSERT INTO
... SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 +++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 ++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 269 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 23 ++
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 512 insertions(+), 162 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 3b435c107d..58730b7dbd 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2294,10 +2297,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 422979dc2e..7a6f9caf8c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -517,6 +517,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -765,12 +779,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1021,12 +1039,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a TransactionId.
* Failure to do this now would result in heap_insert() subsequently
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c04b262f26..a1a10972cb 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 906cab7053..2a3ad42f90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -345,7 +345,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2476203b46..de45aebb3c 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -323,10 +326,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1797,7 +1801,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1845,6 +1962,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2285,96 +2403,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2393,6 +2448,18 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ /*
+ * Generate gather paths according to the added partial paths for the
+ * parallel table-modification command.
+ * Note that true is passed for the "override_rows" parameter, so that
+ * the rows from the cheapest partial path (ModifyTablePath) are used,
+ * not the rel's (possibly estimated) rows.
+ */
+ generate_useful_gather_paths(root, final_rel, true);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7562,7 +7629,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 20e3d263a6..a720227ecc 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -1079,6 +1079,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 69b83071cf..e6af7d3067 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3553,6 +3553,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3563,10 +3564,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3583,47 +3584,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3639,6 +3615,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e31ad6204e..27272ec2c8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index cd1f0d450d..ef651337f2 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 54f4b782fc..cb36413e5b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -267,7 +267,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 7dce70ecea..1197ae24d3 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -598,15 +599,15 @@ typedef struct PartitionedOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v22-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v22-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From f95853059d0592dc81e174c79c3451efc91c680f Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 5 Mar 2021 20:16:09 +1100
Subject: [PATCH v22 1/6] Enable parallel SELECT for "INSERT INTO ... SELECT
...", where it is safe to do so.
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or predicate,
column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for the cases
listed above, for determining whether it is safe to run INSERT in parallel-mode
with an underlying parallel SELECT. The planner is further updated to consider
using parallel SELECT for "INSERT INTO ... SELECT ...", provided nothing unsafe
is found from the additional parallel-safety checks, or from the existing
parallel-safety checks for SELECT.
Prior to entering parallel-mode for execution of INSERT with parallel SELECT,
a TransactionId is acquired and assigned to the current transaction state. This
is necessary to prevent the INSERT from attempting to assign the TransactionId
whilst in parallel-mode, which is not allowed. This approach has a disadvantage
in that if the underlying SELECT does not return any rows, then the
TransactionId is not used, however that shouldn't happen in practice in many
cases.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/transam/xact.c | 26 ++
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 27 +-
src/backend/optimizer/plan/setrefs.c | 13 +
src/backend/optimizer/util/clauses.c | 535 ++++++++++++++++++++++++++-
src/backend/utils/cache/plancache.c | 30 ++
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 3 +-
13 files changed, 645 insertions(+), 15 deletions(-)
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df6b8..422979dc2e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -41,6 +41,7 @@
#include "libpq/be-fsstubs.h"
#include "libpq/pqsignal.h"
#include "miscadmin.h"
+#include "optimizer/optimizer.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "replication/logical.h"
@@ -1014,6 +1015,31 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a TransactionId.
+ * Failure to do this now would result in heap_insert() subsequently
+ * attempting to assign a TransactionId whilst in parallel-mode,
+ * which is not allowed.
+ * This approach has a disadvantage in that if the underlying SELECT
+ * does not return any rows, then the TransactionId is not used,
+ * however that shouldn't happen in practice in many cases.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec2c4..da91cbd2b1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432bfe1..6493a03ff8 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58e86..c5e136e9c3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56bcaf..2476203b46 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +317,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data (except for Insert), or if this is a cursor operation,
+ * or if GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +335,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ is_parallel_allowed_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,9 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+ /* For AcquireExecutorLocks(). */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 42f088ad71..20e3d263a6 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -15,6 +15,7 @@
*/
#include "postgres.h"
+#include "access/xact.h"
#include "access/transam.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
@@ -260,6 +261,18 @@ set_plan_references(PlannerInfo *root, Plan *plan)
*/
add_rtes_to_flat_rtable(root, false);
+ /*
+ * If the Oids of parallel-safety-checked partitions have been recorded,
+ * add them too to glob->relationOids, to register them as plan
+ * dependencies. This is only really needed in the case of a parallel
+ * plan, so that if parallel-unsafe properties are subsequently defined
+ * on the partitions, the cached parallel plan will be invalidated and
+ * a non-parallel plan will be generated.
+ */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ glob->relationOids =
+ list_concat(glob->relationOids, glob->partitionOids);
+
/*
* Adjust RT indexes of PlanRowMarks and add to final rowmarks list
*/
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..3f71c7d60f 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,19 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
+#include "catalog/index.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +49,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +59,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +98,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +111,20 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid,
+ max_parallel_hazard_context *context);
+static bool target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -148,7 +175,6 @@ static Query *substitute_actual_srf_parameters(Query *expr,
static Node *substitute_actual_srf_parameters_mutator(Node *node,
substitute_actual_srf_parameters_context *context);
-
/*****************************************************************************
* Aggregate-function clause manipulation
*****************************************************************************/
@@ -545,14 +571,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +614,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -663,6 +697,11 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
*/
if (IsA(node, CoerceToDomain))
{
+ if (context->target_rte != NULL)
+ {
+ if (target_rel_domain_max_parallel_hazard(((CoerceToDomain *)node)->resulttype, context))
+ return true;
+ }
if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
return true;
}
@@ -687,6 +726,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with
+ * a modifying CTE does not have that flag set, due to a bug in the
+ * query rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -757,6 +817,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +850,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE: 0);
}
/* Recurse to check arguments */
@@ -786,6 +861,460 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ if (trigdesc == NULL)
+ return false;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *)ii_Predicate, context))
+ found_max_hazard = true;
+ }
+ }
+
+ index_close(index_rel, NoLock);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel-safety.
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+/*
+ * target_rel_partitions_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any partitions of a
+ * of a specified relation.
+ */
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+
+ if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+ return false;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ return true;
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ return true;
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ context->command_type,
+ context);
+ table_close(part_rel, NoLock);
+
+ /* Record the partition as a potential plan dependency. */
+ glob->partitionOids =
+ lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_chk_constr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any CHECK expressions or
+ * CHECK constraints related to the specified relation.
+ */
+static bool
+target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+
+ tupdesc = RelationGetDescr(rel);
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or a
+ * temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (target_rel_partitions_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If any triggers exist, check that they are parallel-safe.
+ */
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ return true;
+
+ /*
+ * Column default expressions are only applicable to INSERT and UPDATE.
+ * For columns in the target-list, these are already being checked for
+ * parallel-safety in the max_parallel_hazard() scan of the query tree
+ * in standard_planner(), so there's no need to do it here.
+ * Note that even though column defaults may be specified separately for
+ * each partition in a partitioned table, a partition's default value is
+ * not applied when inserting a tuple through a partitioned table.
+ */
+
+ /*
+ * CHECK constraints are only applicable to INSERT and UPDATE.
+ * If any CHECK constraints exist, determine if they are parallel-safe.
+ */
+ if (target_rel_chk_constr_max_parallel_hazard(rel, context))
+ return true;
+
+ return false;
+}
+
+/*
+ * is_parallel_allowed_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement.
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_allowed_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ if (!IsModifySupportedInParallelMode(parse->commandType))
+ return false;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ * In order to support update, even if only in the leader, some
+ * further work would need to be done. A mechanism would be needed
+ * for sharing combo-cids between leader and workers during
+ * parallel-mode, since for example, the leader might generate a
+ * combo-cid and it needs to be propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel table-modification
+ * operation is not possible (nor desirable).
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a0950489d..084aa85c6e 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,14 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0fc9f..86405a274e 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104d0b..95292d7573 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..8d85b02514 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern bool is_parallel_allowed_for_modify(Query *parse);
#endif /* CLAUSES_H */
--
2.27.0
v22-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-basic-tests-and-doc.patchapplication/octet-stream; name=v22-0002-Parallel-SELECT-for-INSERT-INTO-.-SELECT-basic-tests-and-doc.patchDownload
From 21975923277988683381d0610f019d2cd4516298 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 5 Mar 2021 20:54:50 +1100
Subject: [PATCH v22 2/6] Parallel SELECT for "INSERT INTO ... SELECT ..." -
basic tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 4 +-
src/test/regress/expected/insert_parallel.out | 538 ++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 337 +++++++++++
5 files changed, 880 insertions(+), 1 deletion(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..ef5615cb4b
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,538 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create table dom_table_u (x inotnull_u, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7342..e280198b17 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97324..6a57e889a1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..8bea0dc02f
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,337 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
--
2.27.0
v22-0003-Add-new-parallel-dml-GUC-and-table-options.patchapplication/octet-stream; name=v22-0003-Add-new-parallel-dml-GUC-and-table-options.patchDownload
From a5389e787b4a3f1ff7f4d18a7852a75b9e0b270a Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 21:14:22 +1100
Subject: [PATCH v22 3/6] Add new GUC option "enable_parallel_dml" (boolean)
and table option "parallel_dml_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_dml" is
added, to allow parallel DML to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel dml with finer granularity, to enable/disable the use of parallel
dml for specific tables.
The new table option "parallel_dml_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++
src/backend/access/common/reloptions.c | 25 ++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 ++++++++--
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73596..cc4e0c200e 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-dml" xreflabel="enable_parallel_dml">
+ <term><varname>enable_parallel_dml</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_dml</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ table-modification commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for table-modification. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ table-modification.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..ecb047021d 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_dml_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..5d8f842ddd 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-dml-enabled" xreflabel="parallel_dml_enabled">
+ <term><literal>parallel_dml_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_dml_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel DML for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-dml"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for table-modification. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel DML for this table.
+ Note that if the target table of the parallel DML is partitioned, the
+ <literal>parallel_dml_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..938131a29a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_dml_enabled",
+ "Enables \"parallel dml\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_dml_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_dml_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..c04b262f26 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_dml = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 3f71c7d60f..8205c10803 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1266,8 +1266,10 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
* table-modification statement.
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_dml is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_dml_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1278,12 +1280,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_dml)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1313,7 +1320,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelDML(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3fd1a5fbe2..f5c78258d1 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_dml", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for table-modification commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_dml,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..5c1266ef39 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_dml = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..e7e4af795e 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_dml_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..cd1f0d450d 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_dml;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..7dce70ecea 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelDML
+ * Returns the relation's parallel_dml_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelDML(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_dml_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_dml_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index ef5615cb4b..95efbb9f0e 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +394,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..6c25bc1e2f 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
+ enable_parallel_dml | off
enable_parallel_hash | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 8bea0dc02f..ef03964b5c 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +231,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.27.0
v22-0004-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v22-0004-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From c8b80e9fe90e8bf60e691cde48c0463d33f95a39 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 5 Mar 2021 21:34:47 +1100
Subject: [PATCH v22 4/6] Parallel SELECT for "INSERT INTO ... SELECT ..." -
advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 95efbb9f0e..273f43ce48 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -179,6 +201,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -202,6 +251,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -226,6 +355,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -322,6 +653,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -380,6 +790,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -438,6 +877,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -456,17 +953,25 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -479,19 +984,28 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -502,8 +1016,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -552,15 +1101,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -569,6 +1157,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -576,6 +1199,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -584,4 +1209,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index ef03964b5c..daecbac051 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -139,6 +165,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -150,6 +184,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -223,6 +382,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -255,6 +422,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -269,39 +476,80 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -335,6 +583,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -342,15 +601,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -361,6 +653,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -370,4 +664,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v22-0006-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v22-0006-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 6b9c4a9580f28b68c41fca2e09d5ed95ae5790f0 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 5 Mar 2021 21:58:30 +1100
Subject: [PATCH v22 6/6] INSERT and/or SELECT for "INSERT INTO ... SELECT ..."
- tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 83 +++++++++-
src/test/regress/expected/insert_parallel.out | 142 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 24 +--
3 files changed, 158 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index cec1329e25..039ac18f81 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,14 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
- INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan. Another exception is the command
- <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
- the underlying <literal>SELECT</literal> part of the query.
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
</para>
</listitem>
@@ -426,6 +428,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 273f43ce48..22f379c34f 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -122,14 +122,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -150,7 +150,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -182,15 +182,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -260,9 +260,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -299,9 +299,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -339,9 +339,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -388,9 +388,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -423,9 +423,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -459,9 +459,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -497,9 +497,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -537,9 +537,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -583,21 +583,21 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -626,7 +626,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -662,9 +662,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -724,14 +724,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -763,7 +763,7 @@ explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
(2 rows)
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -792,14 +792,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -858,9 +858,9 @@ alter table parttable1 set (parallel_dml_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -922,9 +922,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -954,7 +954,7 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -975,11 +975,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -1020,7 +1020,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1077,13 +1077,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1110,9 +1110,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index daecbac051..220cb0217e 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -136,7 +136,7 @@ alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -147,7 +147,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -159,7 +159,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -295,14 +295,14 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -310,7 +310,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -362,7 +362,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -375,7 +375,7 @@ truncate testdef;
explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -383,7 +383,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -477,7 +477,7 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan;
+-- (should create a parallel INSERT+SELECT plan;
-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
@@ -527,7 +527,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -573,7 +573,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
On Fri, Mar 5, 2021 at 6:34 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Fri, Mar 5, 2021 at 9:35 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Mar 5, 2021 at 8:24 AM Greg Nancarrow <gregn4422@gmail.com> wrote:
In patch v21-0003-Add-new-parallel-dml-GUC-and-table-options, we are
introducing GUC (enable_parallel_dml) and table option
(parallel_dml_enabled) for this feature. I am a bit worried about
using *_dml in the names because it is quite possible that for
parallel updates and parallel deletes we might not need any such GUC.
The reason we mainly need here is due to checking of parallel-safety
of partitioned tables and updates/deletes handle partitioned tables
differently than inserts so those might not be that costly. It is
possible that they are costly due to a different reason but not sure
mapping those to one GUC or table option is a good idea. Can we
consider using *_insert instead? I think GUC having _insert can be
probably used for a parallel copy (from) as well which I think will
have a similar overhead.I'll need to think about that one.
Okay, one more minor comment on the same patch:
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
I think here the patch is using NoLock based on the assumption that
the relation is already locked in parse/analyze phase, so it's better
to have a comment like the one we have in
target_rel_max_parallel_hazard.
I may be wrong, but I would have thought at least updates would have
similar parallel-safety checking requirements to inserts and would
have similar potential cost issues.
For updates and deletes, we go via inheritance planner where we
already do some work related to partitions, so I think we might not
need to open all the partitions for parallel-safety checks as we do
for inserts. I am sure some part of the code for inserts like the one
we have max_parallel_hazard will be used for updates as well but there
will be probably some change for checking parallel-safety for
partitioned relations.
--
With Regards,
Amit Kapila.
On Fri, Mar 5, 2021 at 6:34 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
For the time being at least, I am posting an updated set of patches,
as I found that the additional parallel-safety checks on DOMAIN check
constraints to be somewhat inefficient and could also be better
integrated into max_parallel_hazard(). I also updated the basic tests
with a test case for this.
Thanks, your changes look good to me. I went ahead and changed the
patch to track the partitionOids once rather than in two separate
lists and use that list in PlanCacheRelCallback to invalidate the
plans. I made few other changes:
a. don't need to retain the lock on indexes as we can't change index expressions
b. added/updated comments at few places in the code.
c. made a few other cosmetic changes
d. ran pgindent
e. slightly modify the commit message.
f. combine the code and test case patch
For now, I have left 0005 and 0006 patches, we can come back to those
once we are done with the first set of patches. The first patch looks
good to me and I think we can commit it and then bikeshed about
GUC/reloption patch.
--
With Regards,
Amit Kapila.
Attachments:
v23-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/x-patch; name=v23-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From a0b7cd32e6aa0337ee54ee5620b8985e47d107e8 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 14:39:35 +0530
Subject: [PATCH v23 1/3] Enable parallel SELECT for "INSERT INTO ... SELECT
...".
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or
predicate, column default expression or check constraint
- Target table has a parallel-unsafe domain constraint on any column
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for
the cases listed above, for determining whether it is safe to run INSERT
in parallel-mode with an underlying parallel SELECT. The planner will
consider using parallel SELECT for "INSERT INTO ... SELECT ...", provided
nothing unsafe is found from the additional parallel-safety checks, or
from the existing parallel-safety checks for SELECT.
While checking parallel-safety, we need to check it for all the partitions
on the table which can be costly especially when we decide not to use a
parallel plan. So, in a separate patch, we will introduce a GUC and or a
reloption to enable/disable parallelism for Insert statements.
Prior to entering parallel-mode for the execution of INSERT with parallel
SELECT, a TransactionId is acquired and assigned to the current
transaction state. This is necessary to prevent the INSERT from attempting
to assign the TransactionId whilst in parallel-mode, which is not allowed.
This approach has a disadvantage in that if the underlying SELECT does not
return any rows, then the TransactionId is not used, however that
shouldn't happen in practice in many cases.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
doc/src/sgml/parallel.sgml | 4 +-
src/backend/access/transam/xact.c | 26 ++
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 37 +-
src/backend/optimizer/util/clauses.c | 552 +++++++++++++++++++++++++-
src/backend/utils/cache/plancache.c | 33 +-
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 3 +-
src/test/regress/expected/insert_parallel.out | 536 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 335 ++++++++++++++++
17 files changed, 1533 insertions(+), 21 deletions(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..cec1329 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df..df6504b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1015,6 +1015,32 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a TransactionId.
+ * Failure to do this now would result in heap_insert() subsequently
+ * attempting to assign a TransactionId whilst in parallel-mode, which
+ * is not allowed.
+ *
+ * This approach has a disadvantage in that if the underlying SELECT
+ * does not return any rows, then the TransactionId is not used,
+ * however that shouldn't happen in practice in many cases.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36..0648dd8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec..da91cbd 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432b..6493a03 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58..c5e136e 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56b..424d25c 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +317,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data (except for Insert), or if this is a cursor operation,
+ * or if GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +335,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ is_parallel_allowed_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,19 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+
+ /*
+ * Register the Oids of parallel-safety-checked partitions as plan
+ * dependencies. This is only really needed in the case of a parallel plan
+ * so that if parallel-unsafe properties are subsequently defined on the
+ * partitions, the cached parallel plan will be invalidated, and a
+ * non-parallel plan will be generated.
+ *
+ * We also use this list to acquire locks on partitions before executing
+ * cached plan. See AcquireExecutorLocks().
+ */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd..f54268c 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,18 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +48,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +58,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +97,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +110,20 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid,
+ max_parallel_hazard_context *context);
+static bool target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -545,14 +571,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +614,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -655,14 +689,20 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
* opclass support functions are generally parallel-safe. XmlExpr is a
* bit more dubious but we can probably get away with it. We err on the
* side of caution by treating CoerceToDomain as parallel-restricted.
- * (Note: in principle that's wrong because a domain constraint could
- * contain a parallel-unsafe function; but useful constraints probably
- * never would have such, and assuming they do would cripple use of
- * parallel query in the presence of domain types.) SQLValueFunction
- * should be safe in all cases. NextValueExpr is parallel-unsafe.
+ * However, for table modification statements, we check the parallel
+ * safety of domain constraints as that could contain a parallel-unsafe
+ * function, and executing that in parallel mode will lead to error.
+ * SQLValueFunction should be safe in all cases. NextValueExpr is
+ * parallel-unsafe.
*/
if (IsA(node, CoerceToDomain))
{
+ if (context->target_rte != NULL)
+ {
+ if (target_rel_domain_max_parallel_hazard(((CoerceToDomain *) node)->resulttype, context))
+ return true;
+ }
+
if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
return true;
}
@@ -688,6 +728,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
/*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with a
+ * modifying CTE does not have that flag set, due to a bug in the query
+ * rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
+ /*
* As a notational convenience for callers, look through RestrictInfo.
*/
else if (IsA(node, RestrictInfo))
@@ -757,6 +818,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +851,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE : 0);
}
/* Recurse to check arguments */
@@ -786,6 +862,468 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or
+ * a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (target_rel_partitions_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If any triggers exist, check that they are parallel-safe.
+ */
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ return true;
+
+ /*
+ * Column default expressions are only applicable to INSERT and UPDATE.
+ * For columns in the target-list, these are already being checked for
+ * parallel-safety in the max_parallel_hazard() scan of the query tree in
+ * standard_planner(), so there's no need to do it here. Note that even
+ * though column defaults may be specified separately for each partition
+ * in a partitioned table, a partition's default value is not applied when
+ * inserting a tuple through a partitioned table.
+ */
+
+ /*
+ * CHECK constraints are only applicable to INSERT and UPDATE. If any
+ * CHECK constraints exist, determine if they are parallel-safe.
+ */
+ if (target_rel_chk_constr_max_parallel_hazard(rel, context))
+ return true;
+
+ return false;
+}
+
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ if (trigdesc == NULL)
+ return false;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *) ii_Predicate, context))
+ found_max_hazard = true;
+ }
+ }
+
+ /*
+ * XXX We don't need to retain lock on index as index expressions
+ * can't be changed later.
+ */
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel-safety.
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_partitions_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any partitions of a
+ * of a specified relation.
+ */
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+
+ if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+ return false;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ return true;
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ return true;
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ context->command_type,
+ context);
+ table_close(part_rel, NoLock);
+
+ /*
+ * Remember partitionOids to record the partition as a potential plan
+ * dependency.
+ */
+ glob->partitionOids = lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_chk_constr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any CHECK expressions or
+ * CHECK constraints related to the specified relation.
+ */
+static bool
+target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+
+ tupdesc = RelationGetDescr(rel);
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_allowed_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement. Currently, we support only Inserts.
+ *
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_allowed_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ if (!IsModifySupportedInParallelMode(parse->commandType))
+ return false;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ *
+ * In order to support update, even if only in the leader, some further
+ * work would need to be done. A mechanism would be needed for sharing
+ * combo-cids between leader and workers during parallel-mode, since for
+ * example, the leader might generate a combo-cid and it needs to be
+ * propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel insert operation is not
+ * desirable.
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a09504..c1f4128 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,14 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
@@ -1990,7 +2020,8 @@ PlanCacheRelCallback(Datum arg, Oid relid)
if (plannedstmt->commandType == CMD_UTILITY)
continue; /* Ignore utility statements */
if ((relid == InvalidOid) ? plannedstmt->relationOids != NIL :
- list_member_oid(plannedstmt->relationOids, relid))
+ (list_member_oid(plannedstmt->relationOids, relid) ||
+ list_member_oid(plannedstmt->partitionOids, relid)))
{
/* Invalidate the generic plan only */
plansource->gplan->is_valid = false;
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b..34cfaf5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0f..86405a2 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104..95292d7 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887..8d85b02 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern bool is_parallel_allowed_for_modify(Query *parse);
#endif /* CLAUSES_H */
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..d5fae79
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,536 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create table dom_table_u (x inotnull_u, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7..e280198 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97..6a57e88 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..70ad31a
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,335 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
--
1.8.3.1
v23-0002-Add-new-GUC-option-enable_parallel_dml-boolean-a.patchapplication/x-patch; name=v23-0002-Add-new-GUC-option-enable_parallel_dml-boolean-a.patchDownload
From 4794c538b8151cabf54a6ce01320e167a1352a1a Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:17:34 +0530
Subject: [PATCH v23 2/3] Add new GUC option "enable_parallel_dml" (boolean)
and table option "parallel_dml_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_dml" is
added, to allow parallel DML to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel dml with finer granularity, to enable/disable the use of parallel
dml for specific tables.
The new table option "parallel_dml_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++++++
src/backend/access/common/reloptions.c | 25 ++++++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 +++++++++++++--
src/backend/utils/misc/guc.c | 11 ++++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73..cc4e0c2 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-dml" xreflabel="enable_parallel_dml">
+ <term><varname>enable_parallel_dml</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_dml</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ table-modification commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for table-modification. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ table-modification.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5a..ecb0470 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_dml_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227..5d8f842 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-dml-enabled" xreflabel="parallel_dml_enabled">
+ <term><literal>parallel_dml_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_dml_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel DML for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-dml"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for table-modification. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel DML for this table.
+ Note that if the target table of the parallel DML is partitioned, the
+ <literal>parallel_dml_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3e..938131a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_dml_enabled",
+ "Enables \"parallel dml\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_dml_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_dml_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_dml_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674..c04b262 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_dml = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f54268c..a0f1f99 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1274,8 +1274,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_dml is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_dml_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1286,12 +1288,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_dml)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1322,7 +1329,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_dml_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_dml_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelDML(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3fd1a5f..f5c7825 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_dml", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for table-modification commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_dml,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528..5c1266e 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_dml = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208a..e7e4af7 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_dml_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be..cd1f0d4 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_dml;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b6398..7dce70e 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -424,6 +425,28 @@ typedef struct ViewOptions
VIEW_OPTION_CHECK_OPTION_CASCADED)
/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_dml_enabled; /* enables planner's use of parallel DML */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelDML
+ * Returns the relation's parallel_dml_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelDML(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_dml_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_dml_enabled) : \
+ (defaultpd))
+
+/*
* RelationIsValid
* True iff relation descriptor is valid.
*/
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79..5dedb72 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +394,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e3..6c25bc1 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
+ enable_parallel_dml | off
enable_parallel_hash | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a..d437ea5 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_dml_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_dml=off and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_dml
+--
+set enable_parallel_dml = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_dml=on and reloption.parallel_dml_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table para_insert_p1 set (parallel_dml_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +231,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_dml_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_dml_enabled
+--
+alter table parttable1 set (parallel_dml_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_dml_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
1.8.3.1
v23-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-.-advan.patchapplication/x-patch; name=v23-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-.-advan.patchDownload
From d35212299ac34a61c4b0893c93bbd8d1c9282232 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH v23 3/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 +++++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5dedb72..f0ac151 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -180,6 +202,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
(1 row)
--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
-- as doing this in a parallel worker would create a new commandId
@@ -203,6 +252,86 @@ select count(*), sum(unique1) from para_insert_f1;
(1 row)
--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
--
@@ -227,6 +356,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
(4 rows)
--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
--
@@ -323,6 +654,51 @@ insert into names7 select * from names order by last_name returning last_name ||
(8 rows)
--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
+--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
-- should create a parallel plan; parallel SELECT)
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -382,6 +792,35 @@ select * from testdef order by a;
truncate testdef;
--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
@@ -439,6 +878,64 @@ select count(*) from parttable1_2;
(1 row)
--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
--
@@ -456,16 +953,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -478,18 +983,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -500,8 +1014,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -551,14 +1100,53 @@ explain (costs off) execute q;
(3 rows)
--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -567,6 +1155,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -574,6 +1197,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -582,4 +1207,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index d437ea5..354baf9 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -140,6 +166,14 @@ explain (costs off) insert into test_data1 select * from test_data where a = 10
insert into test_data1 select * from test_data where a = 10 returning a as data;
--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
-- as doing this in a parallel worker would create a new commandId
@@ -151,6 +185,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
select count(*), sum(unique1) from para_insert_f1;
--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
--
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -224,6 +383,14 @@ select * from testdef order by a;
truncate testdef;
--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
@@ -256,6 +423,46 @@ select count(*) from parttable1_1;
select count(*) from parttable1_2;
--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
--
@@ -269,38 +476,79 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -334,21 +582,65 @@ function make_table_bar();
explain (costs off) execute q;
--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -359,6 +651,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -368,4 +662,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
1.8.3.1
Hi,
Does CATALOG_VERSION_NO need to be bumped (introduction of
partitionOids field) ?
Cheers
On Sat, Mar 6, 2021 at 2:19 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
Show quoted text
On Fri, Mar 5, 2021 at 6:34 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
For the time being at least, I am posting an updated set of patches,
as I found that the additional parallel-safety checks on DOMAIN check
constraints to be somewhat inefficient and could also be better
integrated into max_parallel_hazard(). I also updated the basic tests
with a test case for this.Thanks, your changes look good to me. I went ahead and changed the
patch to track the partitionOids once rather than in two separate
lists and use that list in PlanCacheRelCallback to invalidate the
plans. I made few other changes:
a. don't need to retain the lock on indexes as we can't change index
expressions
b. added/updated comments at few places in the code.
c. made a few other cosmetic changes
d. ran pgindent
e. slightly modify the commit message.
f. combine the code and test case patchFor now, I have left 0005 and 0006 patches, we can come back to those
once we are done with the first set of patches. The first patch looks
good to me and I think we can commit it and then bikeshed about
GUC/reloption patch.--
With Regards,
Amit Kapila.
On Sat, Mar 6, 2021 at 9:13 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Hi,
Does CATALOG_VERSION_NO need to be bumped (introduction of partitionOids field) ?
Good question. I usually update CATALOG_VERSION_NO when the patch
changes any of the system catalogs. This is what is also mentioned in
catversion.h. See the following text in that file: "The catalog
version number is used to flag incompatible changes in the PostgreSQL
system catalogs. Whenever anyone changes the format of a system
catalog relation, or adds, deletes, or modifies standard
catalog entries in such a way that an updated backend wouldn't work
with an old database (or vice versa), the catalog version number
should be changed.".
I might be missing something here but why you think that due to
partitionOids field (in plannedstmt or at another place) we need to
update CATALOG_VERSION_NO?
Anyway, thanks for bringing this up.
--
With Regards,
Amit Kapila.
I was looking at src/backend/nodes/readfuncs.c
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD would call nodeRead() for partitionOids. However, such
field may not exist.
Since there is no 'if (strncmp(":partitionOids", token, length) == 0) {'
check, I was curious whether CATALOG_VERSION_NO should be bumped.
Cheers
On Sat, Mar 6, 2021 at 6:31 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Show quoted text
On Sat, Mar 6, 2021 at 9:13 PM Zhihong Yu <zyu@yugabyte.com> wrote:
Hi,
Does CATALOG_VERSION_NO need to be bumped (introduction of partitionOidsfield) ?
Good question. I usually update CATALOG_VERSION_NO when the patch
changes any of the system catalogs. This is what is also mentioned in
catversion.h. See the following text in that file: "The catalog
version number is used to flag incompatible changes in the PostgreSQL
system catalogs. Whenever anyone changes the format of a system
catalog relation, or adds, deletes, or modifies standard
catalog entries in such a way that an updated backend wouldn't work
with an old database (or vice versa), the catalog version number
should be changed.".I might be missing something here but why you think that due to
partitionOids field (in plannedstmt or at another place) we need to
update CATALOG_VERSION_NO?Anyway, thanks for bringing this up.
--
With Regards,
Amit Kapila.
On Sun, Mar 7, 2021 at 8:24 AM Zhihong Yu <zyu@yugabyte.com> wrote:
I was looking at src/backend/nodes/readfuncs.c
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);READ_NODE_FIELD would call nodeRead() for partitionOids. However, such field may not exist.
Since there is no 'if (strncmp(":partitionOids", token, length) == 0) {' check, I was curious whether CATALOG_VERSION_NO should be bumped.
Won't that be true only if we are reading the stored planned stmt from
disk? But is it possible in any way? The last two commits in this
function (cfffe83b and 45639a05) also didn't bump the catversion.
--
With Regards,
Amit Kapila.
For cfffe83ba82021a1819a656e7ec5c28fb3a99152, if a bool was written (true |
false), READ_INT_FIELD calls atoi() where atoi("true") returns 0
and atoi("false") returns 0 as well.
I am not sure if the new release containing these commits had a higher cat
version compared to the previous release.
If the new releases did have a higher cat version, I guess there was no
issue, by chance.
Cheers
On Sat, Mar 6, 2021 at 8:12 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Show quoted text
On Sun, Mar 7, 2021 at 8:24 AM Zhihong Yu <zyu@yugabyte.com> wrote:
I was looking at src/backend/nodes/readfuncs.c
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);READ_NODE_FIELD would call nodeRead() for partitionOids. However, such
field may not exist.
Since there is no 'if (strncmp(":partitionOids", token, length) == 0) {'
check, I was curious whether CATALOG_VERSION_NO should be bumped.
Won't that be true only if we are reading the stored planned stmt from
disk? But is it possible in any way? The last two commits in this
function (cfffe83b and 45639a05) also didn't bump the catversion.--
With Regards,
Amit Kapila.
For now, I have left 0005 and 0006 patches, we can come back to those once we
are done with the first set of patches. The first patch looks good to me and I
think we can commit it and then bikeshed about GUC/reloption patch.
Hi,
I have a slight comment on 0001 patch.
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update/delete and this isn't supported in a parallel.
IMO, delete on a FK relation will not create new command id, so may be we can just memtion insert/update.
Attaching V24 patchset with the changes:
[0001]: Change the comment memtioned above.
Change the comment memtioned above.
[0002]: Temporarily replaced the option name "*_dml" with "*_insert" and adjusted the associated documents.
Temporarily replaced the option name "*_dml" with "*_insert" and adjusted the associated documents.
Best regards,
houzj
Attachments:
v24-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v24-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From a0b7cd32e6aa0337ee54ee5620b8985e47d107e8 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 14:39:35 +0530
Subject: [PATCH v24 1/3] Enable parallel SELECT for "INSERT INTO ... SELECT
...".
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or
predicate, column default expression or check constraint
- Target table has a parallel-unsafe domain constraint on any column
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for
the cases listed above, for determining whether it is safe to run INSERT
in parallel-mode with an underlying parallel SELECT. The planner will
consider using parallel SELECT for "INSERT INTO ... SELECT ...", provided
nothing unsafe is found from the additional parallel-safety checks, or
from the existing parallel-safety checks for SELECT.
While checking parallel-safety, we need to check it for all the partitions
on the table which can be costly especially when we decide not to use a
parallel plan. So, in a separate patch, we will introduce a GUC and or a
reloption to enable/disable parallelism for Insert statements.
Prior to entering parallel-mode for the execution of INSERT with parallel
SELECT, a TransactionId is acquired and assigned to the current
transaction state. This is necessary to prevent the INSERT from attempting
to assign the TransactionId whilst in parallel-mode, which is not allowed.
This approach has a disadvantage in that if the underlying SELECT does not
return any rows, then the TransactionId is not used, however that
shouldn't happen in practice in many cases.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
doc/src/sgml/parallel.sgml | 4 +-
src/backend/access/transam/xact.c | 26 ++
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 37 +-
src/backend/optimizer/util/clauses.c | 552 +++++++++++++++++++++++++-
src/backend/utils/cache/plancache.c | 33 +-
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 3 +-
src/test/regress/expected/insert_parallel.out | 536 +++++++++++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 335 ++++++++++++++++
17 files changed, 1533 insertions(+), 21 deletions(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff..cec1329 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 4e6a3df..df6504b 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1015,6 +1015,32 @@ IsInParallelMode(void)
}
/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a TransactionId.
+ * Failure to do this now would result in heap_insert() subsequently
+ * attempting to assign a TransactionId whilst in parallel-mode, which
+ * is not allowed.
+ *
+ * This approach has a disadvantage in that if the underlying SELECT
+ * does not return any rows, then the TransactionId is not used,
+ * however that shouldn't happen in practice in many cases.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
+/*
* CommandCounterIncrement
*/
void
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36..0648dd8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec..da91cbd 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432b..6493a03 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58..c5e136e 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56b..424d25c 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +317,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data (except for Insert), or if this is a cursor operation,
+ * or if GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +335,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ is_parallel_allowed_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,19 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+
+ /*
+ * Register the Oids of parallel-safety-checked partitions as plan
+ * dependencies. This is only really needed in the case of a parallel plan
+ * so that if parallel-unsafe properties are subsequently defined on the
+ * partitions, the cached parallel plan will be invalidated, and a
+ * non-parallel plan will be generated.
+ *
+ * We also use this list to acquire locks on partitions before executing
+ * cached plan. See AcquireExecutorLocks().
+ */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd..f54268c 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,18 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +48,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +58,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +97,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +110,20 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid,
+ max_parallel_hazard_context *context);
+static bool target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -545,14 +571,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +614,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -655,14 +689,20 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
* opclass support functions are generally parallel-safe. XmlExpr is a
* bit more dubious but we can probably get away with it. We err on the
* side of caution by treating CoerceToDomain as parallel-restricted.
- * (Note: in principle that's wrong because a domain constraint could
- * contain a parallel-unsafe function; but useful constraints probably
- * never would have such, and assuming they do would cripple use of
- * parallel query in the presence of domain types.) SQLValueFunction
- * should be safe in all cases. NextValueExpr is parallel-unsafe.
+ * However, for table modification statements, we check the parallel
+ * safety of domain constraints as that could contain a parallel-unsafe
+ * function, and executing that in parallel mode will lead to error.
+ * SQLValueFunction should be safe in all cases. NextValueExpr is
+ * parallel-unsafe.
*/
if (IsA(node, CoerceToDomain))
{
+ if (context->target_rte != NULL)
+ {
+ if (target_rel_domain_max_parallel_hazard(((CoerceToDomain *) node)->resulttype, context))
+ return true;
+ }
+
if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
return true;
}
@@ -688,6 +728,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
/*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with a
+ * modifying CTE does not have that flag set, due to a bug in the query
+ * rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
+ /*
* As a notational convenience for callers, look through RestrictInfo.
*/
else if (IsA(node, RestrictInfo))
@@ -757,6 +818,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +851,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE : 0);
}
/* Recurse to check arguments */
@@ -786,6 +862,468 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or
+ * a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (target_rel_partitions_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If any triggers exist, check that they are parallel-safe.
+ */
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ return true;
+
+ /*
+ * Column default expressions are only applicable to INSERT and UPDATE.
+ * For columns in the target-list, these are already being checked for
+ * parallel-safety in the max_parallel_hazard() scan of the query tree in
+ * standard_planner(), so there's no need to do it here. Note that even
+ * though column defaults may be specified separately for each partition
+ * in a partitioned table, a partition's default value is not applied when
+ * inserting a tuple through a partitioned table.
+ */
+
+ /*
+ * CHECK constraints are only applicable to INSERT and UPDATE. If any
+ * CHECK constraints exist, determine if they are parallel-safe.
+ */
+ if (target_rel_chk_constr_max_parallel_hazard(rel, context))
+ return true;
+
+ return false;
+}
+
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ if (trigdesc == NULL)
+ return false;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *) ii_Predicate, context))
+ found_max_hazard = true;
+ }
+ }
+
+ /*
+ * XXX We don't need to retain lock on index as index expressions
+ * can't be changed later.
+ */
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel-safety.
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_partitions_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any partitions of a
+ * of a specified relation.
+ */
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+
+ if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+ return false;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ return true;
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ return true;
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ context->command_type,
+ context);
+ table_close(part_rel, NoLock);
+
+ /*
+ * Remember partitionOids to record the partition as a potential plan
+ * dependency.
+ */
+ glob->partitionOids = lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_chk_constr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any CHECK expressions or
+ * CHECK constraints related to the specified relation.
+ */
+static bool
+target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+
+ tupdesc = RelationGetDescr(rel);
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_allowed_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement. Currently, we support only Inserts.
+ *
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_allowed_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ if (!IsModifySupportedInParallelMode(parse->commandType))
+ return false;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ *
+ * In order to support update, even if only in the leader, some further
+ * work would need to be done. A mechanism would be needed for sharing
+ * combo-cids between leader and workers during parallel-mode, since for
+ * example, the leader might generate a combo-cid and it needs to be
+ * propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel insert operation is not
+ * desirable.
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a09504..c1f4128 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,14 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
@@ -1990,7 +2020,8 @@ PlanCacheRelCallback(Datum arg, Oid relid)
if (plannedstmt->commandType == CMD_UTILITY)
continue; /* Ignore utility statements */
if ((relid == InvalidOid) ? plannedstmt->relationOids != NIL :
- list_member_oid(plannedstmt->relationOids, relid))
+ (list_member_oid(plannedstmt->relationOids, relid) ||
+ list_member_oid(plannedstmt->partitionOids, relid)))
{
/* Invalidate the generic plan only */
plansource->gplan->is_valid = false;
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b..34cfaf5 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0f..86405a2 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104..95292d7 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887..8d85b02 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern bool is_parallel_allowed_for_modify(Query *parse);
#endif /* CLAUSES_H */
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000..d5fae79
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,536 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create table dom_table_u (x inotnull_u, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7..e280198 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97..6a57e88 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000..70ad31a
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,335 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
--
1.8.3.1
v24-0002-Add-new-GUC-option-enable_parallel_insert-boolean-a.patchapplication/octet-stream; name=v24-0002-Add-new-GUC-option-enable_parallel_insert-boolean-a.patchDownload
From 4794c538b8151cabf54a6ce01320e167a1352a1a Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:17:34 +0530
Subject: [PATCH v24 2/3] Add new GUC option "enable_parallel_insert" (boolean)
and table option "parallel_insert_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_insert" is
added, to allow parallel insert to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel insert with finer granularity, to enable/disable the use of parallel
insert for specific tables.
The new table option "parallel_insert_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++++++
src/backend/access/common/reloptions.c | 25 ++++++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 +++++++++++++--
src/backend/utils/misc/guc.c | 11 ++++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73..cc4e0c2 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-insert" xreflabel="enable_parallel_insert">
+ <term><varname>enable_parallel_insert</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_insert</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ <command>INSERT</command> commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for <command>INSERT</command>. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ <command>INSERT</command>.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5a..ecb0470 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_insert_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227..5d8f842 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-insert-enabled" xreflabel="parallel_insert_enabled">
+ <term><literal>parallel_insert_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_insert_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel insert for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for <command>INSERT</command>. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel insert for this table.
+ Note that if the target table of the parallel insert is partitioned, the
+ <literal>parallel_insert_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3e..938131a 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_insert_enabled",
+ "Enables \"parallel insert\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_insert_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_insert_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674..c04b262 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_insert = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f54268c..a0f1f99 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1274,8 +1274,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_insert is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_insert_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1286,12 +1288,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_insert)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1322,7 +1329,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_insert_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_insert_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelInsert(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3fd1a5f..f5c7825 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_insert", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for INSERT commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_insert,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528..5c1266e 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_insert = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208a..e7e4af7 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_insert_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be..cd1f0d4 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_insert;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b6398..7dce70e 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -424,6 +425,28 @@ typedef struct ViewOptions
VIEW_OPTION_CHECK_OPTION_CASCADED)
/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelInsert
+ * Returns the relation's parallel_insert_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelInsert(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_insert_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_insert_enabled) : \
+ (defaultpd))
+
+/*
* RelationIsValid
* True iff relation descriptor is valid.
*/
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79..5dedb72 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +394,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e3..6c25bc1 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -106,6 +106,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
+ enable_parallel_insert | off
enable_partition_pruning | on
enable_partitionwise_aggregate | off
@@ -113,7 +114,7 @@ select name, setting from pg_settings where name like 'enable%';
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a..d437ea5 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +231,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
1.8.3.1
v24-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-.-advan.patchapplication/octet-stream; name=v24-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-.-advan.patchDownload
From d35212299ac34a61c4b0893c93bbd8d1c9282232 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH v24 3/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 +++++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 5dedb72..f0ac151 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -180,6 +202,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
(1 row)
--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
+--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
-- as doing this in a parallel worker would create a new commandId
@@ -203,6 +252,86 @@ select count(*), sum(unique1) from para_insert_f1;
(1 row)
--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
--
@@ -227,6 +356,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
(4 rows)
--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
+--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
--
@@ -323,6 +654,51 @@ insert into names7 select * from names order by last_name returning last_name ||
(8 rows)
--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
+--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
-- should create a parallel plan; parallel SELECT)
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -382,6 +792,35 @@ select * from testdef order by a;
truncate testdef;
--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
+truncate testdef;
+--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
@@ -439,6 +878,64 @@ select count(*) from parttable1_2;
(1 row)
--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
--
@@ -456,16 +953,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -478,18 +983,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -500,8 +1014,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -551,14 +1100,53 @@ explain (costs off) execute q;
(3 rows)
--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
+--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -567,6 +1155,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -574,6 +1197,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -582,4 +1207,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index d437ea5..354baf9 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -140,6 +166,14 @@ explain (costs off) insert into test_data1 select * from test_data where a = 10
insert into test_data1 select * from test_data where a = 10 returning a as data;
--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
+--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
-- as doing this in a parallel worker would create a new commandId
@@ -151,6 +185,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
select count(*), sum(unique1) from para_insert_f1;
--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
+--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
--
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -224,6 +383,14 @@ select * from testdef order by a;
truncate testdef;
--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
@@ -256,6 +423,46 @@ select count(*) from parttable1_1;
select count(*) from parttable1_2;
--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
+--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
--
@@ -269,38 +476,79 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -334,21 +582,65 @@ function make_table_bar();
explain (costs off) execute q;
--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
+--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -359,6 +651,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -368,4 +662,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
1.8.3.1
On Sat, Mar 6, 2021 at 9:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Mar 5, 2021 at 6:34 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
For the time being at least, I am posting an updated set of patches,
as I found that the additional parallel-safety checks on DOMAIN check
constraints to be somewhat inefficient and could also be better
integrated into max_parallel_hazard(). I also updated the basic tests
with a test case for this.Thanks, your changes look good to me. I went ahead and changed the
patch to track the partitionOids once rather than in two separate
lists and use that list in PlanCacheRelCallback to invalidate the
plans. I made few other changes:
a. don't need to retain the lock on indexes as we can't change index expressions
b. added/updated comments at few places in the code.
c. made a few other cosmetic changes
d. ran pgindent
e. slightly modify the commit message.
f. combine the code and test case patchFor now, I have left 0005 and 0006 patches, we can come back to those
once we are done with the first set of patches. The first patch looks
good to me and I think we can commit it and then bikeshed about
GUC/reloption patch.
I've checked and tested the changes, and the resultant patches look OK
to me, so I'm happy for you to commit the first patch.
Regards,
Greg Nancarrow
Fujitsu Australia
From: Amit Kapila <amit.kapila16@gmail.com>
For now, I have left 0005 and 0006 patches, we can come back to those once we
are done with the first set of patches. The first patch looks good to me and I
think we can commit it and then bikeshed about GUC/reloption patch.
Agreed, it looks good to me, too.
Regards
Takayuki Tsunakawa
Hi Amit, Greg,
Sorry, I hadn't noticed last week that some questions were addressed to me.
On Sat, Mar 6, 2021 at 7:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Thanks, your changes look good to me. I went ahead and changed the
patch to track the partitionOids once rather than in two separate
lists and use that list in PlanCacheRelCallback to invalidate the
plans.
Not mixing the partition OIDs into relationOids seems fine to me. I
had considered that option too, but somehow forgot to mention it here.
A couple of things that look odd in v24-0001 (sorry if there were like
that from the beginning):
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid,
+ context->target_rte->rellockmode);
The comment seems to imply that the lock need not be retaken here, but
the code does. Maybe it's fine to pass NoLock here, but use
rellockmode when locking partitions, because they would not have been
locked by the parser/analyzer. Which brings me to:
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
...
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], AccessShareLock);
Not that I can prove there to be any actual hazard, but we tend to
avoid taking locks with different strengths in the same query as would
occur with this piece of code. We're locking the partition with
AccessShareLock here, but the executor will lock the partition with
the stronger RowExclusiveLock mode before trying to insert into it.
Better to use the stronger mode to begin with or just use the target
partitioned table's RTE's rellockmode which would be RowExclusiveLock.
You can see that that's what AcquireExecutorLocksOnPartitions() will
do in the generic plan case.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Mon, Mar 8, 2021 at 12:55 PM Amit Langote <amitlangote09@gmail.com> wrote:
Hi Amit, Greg,
Sorry, I hadn't noticed last week that some questions were addressed to me.
On Sat, Mar 6, 2021 at 7:19 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Thanks, your changes look good to me. I went ahead and changed the
patch to track the partitionOids once rather than in two separate
lists and use that list in PlanCacheRelCallback to invalidate the
plans.Not mixing the partition OIDs into relationOids seems fine to me. I
had considered that option too, but somehow forgot to mention it here.
Okay, thanks for the confirmation.
A couple of things that look odd in v24-0001 (sorry if there were like
that from the beginning):+static bool +target_rel_max_parallel_hazard(max_parallel_hazard_context *context) +{ + bool max_hazard_found; + + Relation targetRel; + + /* + * The target table is already locked by the caller (this is done in the + * parse/analyze phase), and remains locked until end-of-transaction. + */ + targetRel = table_open(context->target_rte->relid, + context->target_rte->rellockmode);The comment seems to imply that the lock need not be retaken here, but
the code does. Maybe it's fine to pass NoLock here, but use
rellockmode when locking partitions, because they would not have been
locked by the parser/analyzer. Which brings me to:+static bool +target_rel_partitions_max_parallel_hazard(Relation rel, + max_parallel_hazard_context *context) +{ ... + for (i = 0; i < pdesc->nparts; i++) + { + bool max_hazard_found; + Relation part_rel; + + /* + * The partition needs to be locked, and remain locked until + * end-of-transaction to ensure its parallel-safety state is not + * hereafter altered. + */ + part_rel = table_open(pdesc->oids[i], AccessShareLock);Not that I can prove there to be any actual hazard, but we tend to
avoid taking locks with different strengths in the same query as would
occur with this piece of code. We're locking the partition with
AccessShareLock here, but the executor will lock the partition with
the stronger RowExclusiveLock mode before trying to insert into it.
Better to use the stronger mode to begin with or just use the target
partitioned table's RTE's rellockmode which would be RowExclusiveLock.
You can see that that's what AcquireExecutorLocksOnPartitions() will
do in the generic plan case.
Both of your comments make sense to me.
--
With Regards,
Amit Kapila.
On Mon, Mar 8, 2021 at 6:25 PM Amit Langote <amitlangote09@gmail.com> wrote:
A couple of things that look odd in v24-0001 (sorry if there were like
that from the beginning):+static bool +target_rel_max_parallel_hazard(max_parallel_hazard_context *context) +{ + bool max_hazard_found; + + Relation targetRel; + + /* + * The target table is already locked by the caller (this is done in the + * parse/analyze phase), and remains locked until end-of-transaction. + */ + targetRel = table_open(context->target_rte->relid, + context->target_rte->rellockmode);The comment seems to imply that the lock need not be retaken here, but
the code does. Maybe it's fine to pass NoLock here, but use
rellockmode when locking partitions, because they would not have been
locked by the parser/analyzer.
Actually, it was originally NoLock, but I think in your suggested
changes (v15_delta.diff) to better integrate the extra parallel-safety
checks into max_parallel_hazard(), you changed "NoLock" to
"context->targetRTE->rellockmode"..
Having said that, since the table is already locked, I think that
using "context->target_rte->rellockmode" in this case just results in
the lock reference count being incremented, so I doubt it really makes
any difference, since it's locked until end-of-transaction.
I'll revert it back to NoLock.
Which brings me to:
+static bool +target_rel_partitions_max_parallel_hazard(Relation rel, + max_parallel_hazard_context *context) +{ ... + for (i = 0; i < pdesc->nparts; i++) + { + bool max_hazard_found; + Relation part_rel; + + /* + * The partition needs to be locked, and remain locked until + * end-of-transaction to ensure its parallel-safety state is not + * hereafter altered. + */ + part_rel = table_open(pdesc->oids[i], AccessShareLock);Not that I can prove there to be any actual hazard, but we tend to
avoid taking locks with different strengths in the same query as would
occur with this piece of code. We're locking the partition with
AccessShareLock here, but the executor will lock the partition with
the stronger RowExclusiveLock mode before trying to insert into it.
Better to use the stronger mode to begin with or just use the target
partitioned table's RTE's rellockmode which would be RowExclusiveLock.
You can see that that's what AcquireExecutorLocksOnPartitions() will
do in the generic plan case.
OK, I see what you mean, best to use the target partitioned table's
RTE's rellockmode in this case then.
I'll update the patch accordingly.
Regards,
Greg Nancarrow
Fujitsu Australia
On Mon, Mar 8, 2021 at 7:42 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Mar 8, 2021 at 6:25 PM Amit Langote <amitlangote09@gmail.com> wrote:
A couple of things that look odd in v24-0001 (sorry if there were like
that from the beginning):+static bool +target_rel_max_parallel_hazard(max_parallel_hazard_context *context) +{ + bool max_hazard_found; + + Relation targetRel; + + /* + * The target table is already locked by the caller (this is done in the + * parse/analyze phase), and remains locked until end-of-transaction. + */ + targetRel = table_open(context->target_rte->relid, + context->target_rte->rellockmode);The comment seems to imply that the lock need not be retaken here, but
the code does. Maybe it's fine to pass NoLock here, but use
rellockmode when locking partitions, because they would not have been
locked by the parser/analyzer.Actually, it was originally NoLock, but I think in your suggested
changes (v15_delta.diff) to better integrate the extra parallel-safety
checks into max_parallel_hazard(), you changed "NoLock" to
"context->targetRTE->rellockmode"..
Having said that, since the table is already locked, I think that
using "context->target_rte->rellockmode" in this case just results in
the lock reference count being incremented, so I doubt it really makes
any difference, since it's locked until end-of-transaction.
I'll revert it back to NoLock.Which brings me to:
+static bool +target_rel_partitions_max_parallel_hazard(Relation rel, + max_parallel_hazard_context *context) +{ ... + for (i = 0; i < pdesc->nparts; i++) + { + bool max_hazard_found; + Relation part_rel; + + /* + * The partition needs to be locked, and remain locked until + * end-of-transaction to ensure its parallel-safety state is not + * hereafter altered. + */ + part_rel = table_open(pdesc->oids[i], AccessShareLock);Not that I can prove there to be any actual hazard, but we tend to
avoid taking locks with different strengths in the same query as would
occur with this piece of code. We're locking the partition with
AccessShareLock here, but the executor will lock the partition with
the stronger RowExclusiveLock mode before trying to insert into it.
Better to use the stronger mode to begin with or just use the target
partitioned table's RTE's rellockmode which would be RowExclusiveLock.
You can see that that's what AcquireExecutorLocksOnPartitions() will
do in the generic plan case.OK, I see what you mean, best to use the target partitioned table's
RTE's rellockmode in this case then.
I'll update the patch accordingly.
I've attached an updated set of patches with the suggested locking changes.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v25-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchapplication/octet-stream; name=v25-0001-Enable-parallel-SELECT-for-INSERT-INTO-.-SELECT.patchDownload
From 8144c956cbc177b0c7aef0dd87e4a79583259396 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Mon, 8 Mar 2021 20:31:28 +1100
Subject: [PATCH v25 1/3] Enable parallel SELECT for "INSERT INTO ... SELECT
...".
Parallel SELECT can't be utilized for INSERT in the following cases:
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table has a parallel-unsafe: trigger, index expression or
predicate, column default expression or check constraint
- Target table has a parallel-unsafe domain constraint on any column
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
The planner is updated to perform additional parallel-safety checks for
the cases listed above, for determining whether it is safe to run INSERT
in parallel-mode with an underlying parallel SELECT. The planner will
consider using parallel SELECT for "INSERT INTO ... SELECT ...", provided
nothing unsafe is found from the additional parallel-safety checks, or
from the existing parallel-safety checks for SELECT.
While checking parallel-safety, we need to check it for all the partitions
on the table which can be costly especially when we decide not to use a
parallel plan. So, in a separate patch, we will introduce a GUC and or a
reloption to enable/disable parallelism for Insert statements.
Prior to entering parallel-mode for the execution of INSERT with parallel
SELECT, a TransactionId is acquired and assigned to the current
transaction state. This is necessary to prevent the INSERT from attempting
to assign the TransactionId whilst in parallel-mode, which is not allowed.
This approach has a disadvantage in that if the underlying SELECT does not
return any rows, then the TransactionId is not used, however that
shouldn't happen in practice in many cases.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
doc/src/sgml/parallel.sgml | 4 +-
src/backend/access/transam/xact.c | 26 +
src/backend/executor/execMain.c | 3 +
src/backend/nodes/copyfuncs.c | 1 +
src/backend/nodes/outfuncs.c | 2 +
src/backend/nodes/readfuncs.c | 1 +
src/backend/optimizer/plan/planner.c | 37 +-
src/backend/optimizer/util/clauses.c | 550 +++++++++++++++++-
src/backend/utils/cache/plancache.c | 33 +-
src/include/access/xact.h | 15 +
src/include/nodes/pathnodes.h | 2 +
src/include/nodes/plannodes.h | 2 +
src/include/optimizer/clauses.h | 3 +-
src/test/regress/expected/insert_parallel.out | 536 +++++++++++++++++
src/test/regress/parallel_schedule | 1 +
src/test/regress/serial_schedule | 1 +
src/test/regress/sql/insert_parallel.sql | 335 +++++++++++
17 files changed, 1531 insertions(+), 21 deletions(-)
create mode 100644 src/test/regress/expected/insert_parallel.out
create mode 100644 src/test/regress/sql/insert_parallel.sql
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index c81abff48d..cec1329e25 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -146,7 +146,9 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
a CTE, no parallel plans for that query will be generated. As an
exception, the commands <literal>CREATE TABLE ... AS</literal>, <literal>SELECT
INTO</literal>, and <literal>CREATE MATERIALIZED VIEW</literal> which create a new
- table and populate it can use a parallel plan.
+ table and populate it can use a parallel plan. Another exception is the command
+ <literal>INSERT INTO ... SELECT ...</literal> which can use a parallel plan for
+ the underlying <literal>SELECT</literal> part of the query.
</para>
</listitem>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index c83aa16f2c..6395a9b240 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1014,6 +1014,32 @@ IsInParallelMode(void)
return CurrentTransactionState->parallelModeLevel != 0;
}
+/*
+ * PrepareParallelModePlanExec
+ *
+ * Prepare for entering parallel mode plan execution, based on command-type.
+ */
+void
+PrepareParallelModePlanExec(CmdType commandType)
+{
+ if (IsModifySupportedInParallelMode(commandType))
+ {
+ Assert(!IsInParallelMode());
+
+ /*
+ * Prepare for entering parallel mode by assigning a TransactionId.
+ * Failure to do this now would result in heap_insert() subsequently
+ * attempting to assign a TransactionId whilst in parallel-mode, which
+ * is not allowed.
+ *
+ * This approach has a disadvantage in that if the underlying SELECT
+ * does not return any rows, then the TransactionId is not used,
+ * however that shouldn't happen in practice in many cases.
+ */
+ (void) GetCurrentTransactionId();
+ }
+}
+
/*
* CommandCounterIncrement
*/
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index c74ce36ffb..0648dd82ba 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -1512,7 +1512,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
+ {
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
EnterParallelMode();
+ }
/*
* Loop until we've processed the proper number of tuples from the plan.
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index aaba1ec2c4..da91cbd2b1 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -96,6 +96,7 @@ _copyPlannedStmt(const PlannedStmt *from)
COPY_BITMAPSET_FIELD(rewindPlanIDs);
COPY_NODE_FIELD(rowMarks);
COPY_NODE_FIELD(relationOids);
+ COPY_NODE_FIELD(partitionOids);
COPY_NODE_FIELD(invalItems);
COPY_NODE_FIELD(paramExecTypes);
COPY_NODE_FIELD(utilityStmt);
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 8fc432bfe1..6493a03ff8 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -314,6 +314,7 @@ _outPlannedStmt(StringInfo str, const PlannedStmt *node)
WRITE_BITMAPSET_FIELD(rewindPlanIDs);
WRITE_NODE_FIELD(rowMarks);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_NODE_FIELD(utilityStmt);
@@ -2221,6 +2222,7 @@ _outPlannerGlobal(StringInfo str, const PlannerGlobal *node)
WRITE_NODE_FIELD(resultRelations);
WRITE_NODE_FIELD(appendRelations);
WRITE_NODE_FIELD(relationOids);
+ WRITE_NODE_FIELD(partitionOids);
WRITE_NODE_FIELD(invalItems);
WRITE_NODE_FIELD(paramExecTypes);
WRITE_UINT_FIELD(lastPHId);
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 718fb58e86..c5e136e9c3 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1590,6 +1590,7 @@ _readPlannedStmt(void)
READ_BITMAPSET_FIELD(rewindPlanIDs);
READ_NODE_FIELD(rowMarks);
READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(partitionOids);
READ_NODE_FIELD(invalItems);
READ_NODE_FIELD(paramExecTypes);
READ_NODE_FIELD(utilityStmt);
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 545b56bcaf..424d25cbd5 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -305,6 +305,7 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
glob->resultRelations = NIL;
glob->appendRelations = NIL;
glob->relationOids = NIL;
+ glob->partitionOids = NIL;
glob->invalItems = NIL;
glob->paramExecTypes = NIL;
glob->lastPHId = 0;
@@ -316,16 +317,16 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
/*
* Assess whether it's feasible to use parallel mode for this query. We
* can't do this in a standalone backend, or if the command will try to
- * modify any data, or if this is a cursor operation, or if GUCs are set
- * to values that don't permit parallelism, or if parallel-unsafe
- * functions are present in the query tree.
+ * modify any data (except for Insert), or if this is a cursor operation,
+ * or if GUCs are set to values that don't permit parallelism, or if
+ * parallel-unsafe functions are present in the query tree.
*
- * (Note that we do allow CREATE TABLE AS, SELECT INTO, and CREATE
- * MATERIALIZED VIEW to use parallel plans, but as of now, only the leader
- * backend writes into a completely new table. In the future, we can
- * extend it to allow workers to write into the table. However, to allow
- * parallel updates and deletes, we have to solve other problems,
- * especially around combo CIDs.)
+ * (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
+ * INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
+ * of now, only the leader backend writes into a completely new table. In
+ * the future, we can extend it to allow workers to write into the table.
+ * However, to allow parallel updates and deletes, we have to solve other
+ * problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -334,13 +335,14 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*/
if ((cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
IsUnderPostmaster &&
- parse->commandType == CMD_SELECT &&
+ (parse->commandType == CMD_SELECT ||
+ is_parallel_allowed_for_modify(parse)) &&
!parse->hasModifyingCTE &&
max_parallel_workers_per_gather > 0 &&
!IsParallelWorker())
{
/* all the cheap tests pass, so scan the query tree */
- glob->maxParallelHazard = max_parallel_hazard(parse);
+ glob->maxParallelHazard = max_parallel_hazard(parse, glob);
glob->parallelModeOK = (glob->maxParallelHazard != PROPARALLEL_UNSAFE);
}
else
@@ -521,6 +523,19 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
result->rewindPlanIDs = glob->rewindPlanIDs;
result->rowMarks = glob->finalrowmarks;
result->relationOids = glob->relationOids;
+
+ /*
+ * Register the Oids of parallel-safety-checked partitions as plan
+ * dependencies. This is only really needed in the case of a parallel plan
+ * so that if parallel-unsafe properties are subsequently defined on the
+ * partitions, the cached parallel plan will be invalidated, and a
+ * non-parallel plan will be generated.
+ *
+ * We also use this list to acquire locks on partitions before executing
+ * cached plan. See AcquireExecutorLocks().
+ */
+ if (glob->partitionOids != NIL && glob->parallelModeNeeded)
+ result->partitionOids = glob->partitionOids;
result->invalItems = glob->invalItems;
result->paramExecTypes = glob->paramExecTypes;
/* utilityStmt should be null, but we might as well copy it */
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index f3786dd2b6..c416263749 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -19,13 +19,18 @@
#include "postgres.h"
+#include "access/genam.h"
#include "access/htup_details.h"
+#include "access/table.h"
+#include "access/xact.h"
#include "catalog/pg_aggregate.h"
#include "catalog/pg_class.h"
+#include "catalog/pg_constraint.h"
#include "catalog/pg_language.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
+#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "funcapi.h"
@@ -43,6 +48,8 @@
#include "parser/parse_agg.h"
#include "parser/parse_coerce.h"
#include "parser/parse_func.h"
+#include "parser/parsetree.h"
+#include "partitioning/partdesc.h"
#include "rewrite/rewriteManip.h"
#include "tcop/tcopprot.h"
#include "utils/acl.h"
@@ -51,6 +58,8 @@
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
+#include "utils/partcache.h"
+#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/typcache.h"
@@ -88,6 +97,9 @@ typedef struct
char max_hazard; /* worst proparallel hazard found so far */
char max_interesting; /* worst proparallel hazard of interest */
List *safe_param_ids; /* PARAM_EXEC Param IDs to treat as safe */
+ RangeTblEntry *target_rte; /* query's target relation if any */
+ CmdType command_type; /* query's command type */
+ PlannerGlobal *planner_global; /* global info for planner invocation */
} max_parallel_hazard_context;
static bool contain_agg_clause_walker(Node *node, void *context);
@@ -98,6 +110,20 @@ static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_not_nextval_walker(Node *node, void *context);
static bool max_parallel_hazard_walker(Node *node,
max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context);
+static bool target_rel_max_parallel_hazard_recurse(Relation relation,
+ CmdType command_type,
+ max_parallel_hazard_context *context);
+static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context);
+static bool target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_domain_max_parallel_hazard(Oid typid,
+ max_parallel_hazard_context *context);
+static bool target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
+static bool target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static bool contain_exec_param_walker(Node *node, List *param_ids);
static bool contain_context_dependent_node(Node *clause);
@@ -545,14 +571,19 @@ contain_volatile_functions_not_nextval_walker(Node *node, void *context)
* later, in the common case where everything is SAFE.
*/
char
-max_parallel_hazard(Query *parse)
+max_parallel_hazard(Query *parse, PlannerGlobal *glob)
{
max_parallel_hazard_context context;
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_UNSAFE;
context.safe_param_ids = NIL;
+ context.target_rte = parse->resultRelation > 0 ?
+ rt_fetch(parse->resultRelation, parse->rtable) : NULL;
+ context.command_type = parse->commandType;
+ context.planner_global = glob;
(void) max_parallel_hazard_walker((Node *) parse, &context);
+
return context.max_hazard;
}
@@ -583,6 +614,9 @@ is_parallel_safe(PlannerInfo *root, Node *node)
context.max_hazard = PROPARALLEL_SAFE;
context.max_interesting = PROPARALLEL_RESTRICTED;
context.safe_param_ids = NIL;
+ context.command_type = node != NULL && IsA(node, Query) ?
+ castNode(Query, node)->commandType : CMD_UNKNOWN;
+ context.planner_global = root->glob;
/*
* The params that refer to the same or parent query level are considered
@@ -655,14 +689,20 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
* opclass support functions are generally parallel-safe. XmlExpr is a
* bit more dubious but we can probably get away with it. We err on the
* side of caution by treating CoerceToDomain as parallel-restricted.
- * (Note: in principle that's wrong because a domain constraint could
- * contain a parallel-unsafe function; but useful constraints probably
- * never would have such, and assuming they do would cripple use of
- * parallel query in the presence of domain types.) SQLValueFunction
- * should be safe in all cases. NextValueExpr is parallel-unsafe.
+ * However, for table modification statements, we check the parallel
+ * safety of domain constraints as that could contain a parallel-unsafe
+ * function, and executing that in parallel mode will lead to error.
+ * SQLValueFunction should be safe in all cases. NextValueExpr is
+ * parallel-unsafe.
*/
if (IsA(node, CoerceToDomain))
{
+ if (context->target_rte != NULL)
+ {
+ if (target_rel_domain_max_parallel_hazard(((CoerceToDomain *) node)->resulttype, context))
+ return true;
+ }
+
if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
return true;
}
@@ -687,6 +727,27 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
return true;
}
+ /*
+ * ModifyingCTE expressions are treated as parallel-unsafe.
+ *
+ * XXX Normally, if the Query has a modifying CTE, the hasModifyingCTE
+ * flag is set in the Query tree, and the query will be regarded as
+ * parallel-usafe. However, in some cases, a re-written query with a
+ * modifying CTE does not have that flag set, due to a bug in the query
+ * rewriter.
+ */
+ else if (IsA(node, CommonTableExpr))
+ {
+ CommonTableExpr *cte = (CommonTableExpr *) node;
+ Query *ctequery = castNode(Query, cte->ctequery);
+
+ if (ctequery->commandType != CMD_SELECT)
+ {
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ return true;
+ }
+ }
+
/*
* As a notational convenience for callers, look through RestrictInfo.
*/
@@ -757,6 +818,19 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
}
return false; /* nothing to recurse to */
}
+ else if (IsA(node, RangeTblEntry))
+ {
+ RangeTblEntry *rte = (RangeTblEntry *) node;
+
+ /* Nothing interesting to check for SELECTs */
+ if (context->target_rte == NULL)
+ return false;
+
+ if (rte == context->target_rte)
+ return target_rel_max_parallel_hazard(context);
+
+ return false;
+ }
/*
* When we're first invoked on a completely unplanned tree, we must
@@ -777,7 +851,9 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
/* Recurse into subselects */
return query_tree_walker(query,
max_parallel_hazard_walker,
- context, 0);
+ context,
+ context->target_rte != NULL ?
+ QTW_EXAMINE_RTES_BEFORE : 0);
}
/* Recurse to check arguments */
@@ -786,6 +862,466 @@ max_parallel_hazard_walker(Node *node, max_parallel_hazard_context *context)
context);
}
+/*
+ * target_rel_max_parallel_hazard
+ *
+ * Determines the maximum parallel-mode hazard level for modification
+ * of a specified relation.
+ */
+static bool
+target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
+{
+ bool max_hazard_found;
+
+ Relation targetRel;
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ targetRel = table_open(context->target_rte->relid, NoLock);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(targetRel,
+ context->command_type,
+ context);
+ table_close(targetRel, NoLock);
+
+ return max_hazard_found;
+}
+
+static bool
+target_rel_max_parallel_hazard_recurse(Relation rel,
+ CmdType command_type,
+ max_parallel_hazard_context *context)
+{
+ /* Currently only CMD_INSERT is supported */
+ Assert(command_type == CMD_INSERT);
+
+ /*
+ * We can't support table modification in a parallel worker if it's a
+ * foreign table/partition (no FDW API for supporting parallel access) or
+ * a temporary table.
+ */
+ if (rel->rd_rel->relkind == RELKIND_FOREIGN_TABLE ||
+ RelationUsesLocalBuffers(rel))
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+
+ /*
+ * If a partitioned table, check that each partition is safe for
+ * modification in parallel-mode.
+ */
+ if (target_rel_partitions_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If there are any index expressions or index predicate, check that they
+ * are parallel-mode safe.
+ */
+ if (target_rel_index_max_parallel_hazard(rel, context))
+ return true;
+
+ /*
+ * If any triggers exist, check that they are parallel-safe.
+ */
+ if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ return true;
+
+ /*
+ * Column default expressions are only applicable to INSERT and UPDATE.
+ * For columns in the target-list, these are already being checked for
+ * parallel-safety in the max_parallel_hazard() scan of the query tree in
+ * standard_planner(), so there's no need to do it here. Note that even
+ * though column defaults may be specified separately for each partition
+ * in a partitioned table, a partition's default value is not applied when
+ * inserting a tuple through a partitioned table.
+ */
+
+ /*
+ * CHECK constraints are only applicable to INSERT and UPDATE. If any
+ * CHECK constraints exist, determine if they are parallel-safe.
+ */
+ if (target_rel_chk_constr_max_parallel_hazard(rel, context))
+ return true;
+
+ return false;
+}
+
+/*
+ * target_rel_trigger_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ */
+static bool
+target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+ max_parallel_hazard_context *context)
+{
+ int i;
+
+ if (trigdesc == NULL)
+ return false;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ return true;
+
+ /*
+ * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
+ * the relation, and this would result in creation of new CommandIds
+ * on insert/update and this isn't supported in a parallel
+ * worker (but is safe in the parallel leader).
+ */
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ if (trigtype == RI_TRIGGER_FK)
+ {
+ if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_index_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any existing index
+ * expressions or index predicate of a specified relation.
+ */
+static bool
+target_rel_index_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ List *index_oid_list;
+ ListCell *lc;
+ bool found_max_hazard = false;
+ LOCKMODE lockmode = AccessShareLock;
+
+ index_oid_list = RelationGetIndexList(rel);
+ foreach(lc, index_oid_list)
+ {
+ Relation index_rel;
+ Form_pg_index indexStruct;
+ List *ii_Expressions;
+ List *ii_Predicate;
+ Oid index_oid = lfirst_oid(lc);
+
+ index_rel = index_open(index_oid, lockmode);
+
+ indexStruct = index_rel->rd_index;
+ ii_Expressions = RelationGetIndexExpressions(index_rel);
+
+ if (ii_Expressions != NIL)
+ {
+ int i;
+ ListCell *index_expr_item = list_head(ii_Expressions);
+
+ for (i = 0; i < indexStruct->indnatts; i++)
+ {
+ int keycol = indexStruct->indkey.values[i];
+
+ if (keycol == 0)
+ {
+ /* Found an index expression */
+
+ Node *index_expr;
+
+ Assert(index_expr_item != NULL);
+ if (index_expr_item == NULL) /* shouldn't happen */
+ {
+ elog(WARNING, "too few entries in indexprs list");
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr = (Node *) lfirst(index_expr_item);
+
+ if (max_parallel_hazard_walker(index_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+
+ index_expr_item = lnext(ii_Expressions, index_expr_item);
+ }
+ }
+ }
+
+ if (!found_max_hazard)
+ {
+ ii_Predicate = RelationGetIndexPredicate(index_rel);
+ if (ii_Predicate != NIL)
+ {
+ if (max_parallel_hazard_walker((Node *) ii_Predicate, context))
+ found_max_hazard = true;
+ }
+ }
+
+ /*
+ * XXX We don't need to retain lock on index as index expressions
+ * can't be changed later.
+ */
+ index_close(index_rel, lockmode);
+ }
+ list_free(index_oid_list);
+
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_domain_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for the specified DOMAIN type.
+ * Only any CHECK expressions are examined for parallel-safety.
+ */
+static bool
+target_rel_domain_max_parallel_hazard(Oid typid, max_parallel_hazard_context *context)
+{
+ Relation con_rel;
+ ScanKeyData key[1];
+ SysScanDesc scan;
+ HeapTuple tup;
+ bool found_max_hazard = false;
+
+ LOCKMODE lockmode = AccessShareLock;
+
+ con_rel = table_open(ConstraintRelationId, lockmode);
+
+ ScanKeyInit(&key[0],
+ Anum_pg_constraint_contypid, BTEqualStrategyNumber,
+ F_OIDEQ, ObjectIdGetDatum(typid));
+ scan = systable_beginscan(con_rel, ConstraintTypidIndexId, true,
+ NULL, 1, key);
+
+ while (HeapTupleIsValid((tup = systable_getnext(scan))))
+ {
+ Form_pg_constraint con = (Form_pg_constraint) GETSTRUCT(tup);
+
+ if (con->contype == CONSTRAINT_CHECK)
+ {
+ char *conbin;
+ Datum val;
+ bool isnull;
+ Expr *check_expr;
+
+ val = SysCacheGetAttr(CONSTROID, tup,
+ Anum_pg_constraint_conbin, &isnull);
+ Assert(!isnull);
+ if (isnull)
+ {
+ /*
+ * This shouldn't ever happen, but if it does, log a WARNING
+ * and return UNSAFE, rather than erroring out.
+ */
+ elog(WARNING, "null conbin for constraint %u", con->oid);
+ context->max_hazard = PROPARALLEL_UNSAFE;
+ found_max_hazard = true;
+ break;
+ }
+ conbin = TextDatumGetCString(val);
+ check_expr = stringToNode(conbin);
+ pfree(conbin);
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ {
+ found_max_hazard = true;
+ break;
+ }
+ }
+ }
+
+ systable_endscan(scan);
+ table_close(con_rel, lockmode);
+ return found_max_hazard;
+}
+
+/*
+ * target_rel_partitions_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any partitions of a
+ * of a specified relation.
+ */
+static bool
+target_rel_partitions_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ int i;
+ PartitionDesc pdesc;
+ PartitionKey pkey;
+ ListCell *partexprs_item;
+ int partnatts;
+ List *partexprs;
+ PlannerGlobal *glob;
+
+
+ if (rel->rd_rel->relkind != RELKIND_PARTITIONED_TABLE)
+ return false;
+
+ pkey = RelationGetPartitionKey(rel);
+
+ partnatts = get_partition_natts(pkey);
+ partexprs = get_partition_exprs(pkey);
+
+ partexprs_item = list_head(partexprs);
+ for (i = 0; i < partnatts; i++)
+ {
+ /* Check parallel-safety of partition key support functions */
+ if (OidIsValid(pkey->partsupfunc[i].fn_oid))
+ {
+ if (max_parallel_hazard_test(func_parallel(pkey->partsupfunc[i].fn_oid), context))
+ return true;
+ }
+
+ /* Check parallel-safety of any expressions in the partition key */
+ if (get_partition_col_attnum(pkey, i) == 0)
+ {
+ Node *check_expr = (Node *) lfirst(partexprs_item);
+
+ if (max_parallel_hazard_walker(check_expr, context))
+ return true;
+
+ partexprs_item = lnext(partexprs, partexprs_item);
+ }
+ }
+
+ /* Recursively check each partition ... */
+
+ /* Create the PartitionDirectory infrastructure if we didn't already */
+ glob = context->planner_global;
+ if (glob->partition_directory == NULL)
+ glob->partition_directory =
+ CreatePartitionDirectory(CurrentMemoryContext);
+
+ pdesc = PartitionDirectoryLookup(glob->partition_directory, rel);
+
+ for (i = 0; i < pdesc->nparts; i++)
+ {
+ bool max_hazard_found;
+ Relation part_rel;
+
+ /*
+ * The partition needs to be locked, and remain locked until
+ * end-of-transaction to ensure its parallel-safety state is not
+ * hereafter altered.
+ */
+ part_rel = table_open(pdesc->oids[i], context->target_rte->rellockmode);
+ max_hazard_found = target_rel_max_parallel_hazard_recurse(part_rel,
+ context->command_type,
+ context);
+ table_close(part_rel, NoLock);
+
+ /*
+ * Remember partitionOids to record the partition as a potential plan
+ * dependency.
+ */
+ glob->partitionOids = lappend_oid(glob->partitionOids, pdesc->oids[i]);
+
+ if (max_hazard_found)
+ return true;
+ }
+
+ return false;
+}
+
+/*
+ * target_rel_chk_constr_max_parallel_hazard
+ *
+ * Finds the maximum parallel-mode hazard level for any CHECK expressions or
+ * CHECK constraints related to the specified relation.
+ */
+static bool
+target_rel_chk_constr_max_parallel_hazard(Relation rel,
+ max_parallel_hazard_context *context)
+{
+ TupleDesc tupdesc;
+
+ tupdesc = RelationGetDescr(rel);
+
+ /*
+ * Determine if there are any CHECK constraints which are not
+ * parallel-safe.
+ */
+ if (tupdesc->constr != NULL && tupdesc->constr->num_check > 0)
+ {
+ int i;
+
+ ConstrCheck *check = tupdesc->constr->check;
+
+ for (i = 0; i < tupdesc->constr->num_check; i++)
+ {
+ Expr *check_expr = stringToNode(check->ccbin);
+
+ if (max_parallel_hazard_walker((Node *) check_expr, context))
+ return true;
+ }
+ }
+
+ return false;
+}
+
+/*
+ * is_parallel_allowed_for_modify
+ *
+ * Check at a high-level if parallel mode is able to be used for the specified
+ * table-modification statement. Currently, we support only Inserts.
+ *
+ * It's not possible in the following cases:
+ *
+ * 1) INSERT...ON CONFLICT...DO UPDATE
+ * 2) INSERT without SELECT
+ *
+ * (Note: we don't do in-depth parallel-safety checks here, we do only the
+ * cheaper tests that can quickly exclude obvious cases for which
+ * parallelism isn't supported, to avoid having to do further parallel-safety
+ * checks for these)
+ */
+bool
+is_parallel_allowed_for_modify(Query *parse)
+{
+ bool hasSubQuery;
+ RangeTblEntry *rte;
+ ListCell *lc;
+
+ if (!IsModifySupportedInParallelMode(parse->commandType))
+ return false;
+
+ /*
+ * UPDATE is not currently supported in parallel-mode, so prohibit
+ * INSERT...ON CONFLICT...DO UPDATE...
+ *
+ * In order to support update, even if only in the leader, some further
+ * work would need to be done. A mechanism would be needed for sharing
+ * combo-cids between leader and workers during parallel-mode, since for
+ * example, the leader might generate a combo-cid and it needs to be
+ * propagated to the workers.
+ */
+ if (parse->commandType == CMD_INSERT &&
+ parse->onConflict != NULL &&
+ parse->onConflict->action == ONCONFLICT_UPDATE)
+ return false;
+
+ /*
+ * If there is no underlying SELECT, a parallel insert operation is not
+ * desirable.
+ */
+ hasSubQuery = false;
+ foreach(lc, parse->rtable)
+ {
+ rte = lfirst_node(RangeTblEntry, lc);
+ if (rte->rtekind == RTE_SUBQUERY)
+ {
+ hasSubQuery = true;
+ break;
+ }
+ }
+
+ return hasSubQuery;
+}
/*****************************************************************************
* Check clauses for nonstrict functions
diff --git a/src/backend/utils/cache/plancache.c b/src/backend/utils/cache/plancache.c
index 1a0950489d..c1f4128445 100644
--- a/src/backend/utils/cache/plancache.c
+++ b/src/backend/utils/cache/plancache.c
@@ -1735,6 +1735,23 @@ QueryListGetPrimaryStmt(List *stmts)
return NULL;
}
+static void
+AcquireExecutorLocksOnPartitions(List *partitionOids, int lockmode,
+ bool acquire)
+{
+ ListCell *lc;
+
+ foreach(lc, partitionOids)
+ {
+ Oid partOid = lfirst_oid(lc);
+
+ if (acquire)
+ LockRelationOid(partOid, lockmode);
+ else
+ UnlockRelationOid(partOid, lockmode);
+ }
+}
+
/*
* AcquireExecutorLocks: acquire locks needed for execution of a cached plan;
* or release them if acquire is false.
@@ -1748,6 +1765,8 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
{
PlannedStmt *plannedstmt = lfirst_node(PlannedStmt, lc1);
ListCell *lc2;
+ Index rti,
+ resultRelation = 0;
if (plannedstmt->commandType == CMD_UTILITY)
{
@@ -1765,6 +1784,9 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
continue;
}
+ rti = 1;
+ if (plannedstmt->resultRelations)
+ resultRelation = linitial_int(plannedstmt->resultRelations);
foreach(lc2, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(lc2);
@@ -1782,6 +1804,14 @@ AcquireExecutorLocks(List *stmt_list, bool acquire)
LockRelationOid(rte->relid, rte->rellockmode);
else
UnlockRelationOid(rte->relid, rte->rellockmode);
+
+ /* Lock partitions ahead of modifying them in parallel mode. */
+ if (rti == resultRelation &&
+ plannedstmt->partitionOids != NIL)
+ AcquireExecutorLocksOnPartitions(plannedstmt->partitionOids,
+ rte->rellockmode, acquire);
+
+ rti++;
}
}
}
@@ -1990,7 +2020,8 @@ PlanCacheRelCallback(Datum arg, Oid relid)
if (plannedstmt->commandType == CMD_UTILITY)
continue; /* Ignore utility statements */
if ((relid == InvalidOid) ? plannedstmt->relationOids != NIL :
- list_member_oid(plannedstmt->relationOids, relid))
+ (list_member_oid(plannedstmt->relationOids, relid) ||
+ list_member_oid(plannedstmt->partitionOids, relid)))
{
/* Invalidate the generic plan only */
plansource->gplan->is_valid = false;
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index f49a57b35e..34cfaf542c 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -466,5 +466,20 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
+extern void PrepareParallelModePlanExec(CmdType commandType);
+
+/*
+ * IsModifySupportedInParallelMode
+ *
+ * Indicates whether execution of the specified table-modification command
+ * (INSERT/UPDATE/DELETE) in parallel-mode is supported, subject to certain
+ * parallel-safety conditions.
+ */
+static inline bool
+IsModifySupportedInParallelMode(CmdType commandType)
+{
+ /* Currently only INSERT is supported */
+ return (commandType == CMD_INSERT);
+}
#endif /* XACT_H */
diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h
index b8a6e0fc9f..86405a274e 100644
--- a/src/include/nodes/pathnodes.h
+++ b/src/include/nodes/pathnodes.h
@@ -120,6 +120,8 @@ typedef struct PlannerGlobal
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 6e62104d0b..95292d7573 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -79,6 +79,8 @@ typedef struct PlannedStmt
List *relationOids; /* OIDs of relations the plan depends on */
+ List *partitionOids; /* OIDs of partitions the plan depends on */
+
List *invalItems; /* other dependencies, as PlanInvalItems */
List *paramExecTypes; /* type OIDs for PARAM_EXEC Params */
diff --git a/src/include/optimizer/clauses.h b/src/include/optimizer/clauses.h
index 0673887a85..8d85b02514 100644
--- a/src/include/optimizer/clauses.h
+++ b/src/include/optimizer/clauses.h
@@ -32,7 +32,7 @@ extern double expression_returns_set_rows(PlannerInfo *root, Node *clause);
extern bool contain_subplans(Node *clause);
-extern char max_parallel_hazard(Query *parse);
+extern char max_parallel_hazard(Query *parse, PlannerGlobal *glob);
extern bool is_parallel_safe(PlannerInfo *root, Node *node);
extern bool contain_nonstrict_functions(Node *clause);
extern bool contain_exec_param(Node *clause, List *param_ids);
@@ -52,5 +52,6 @@ extern void CommuteOpExpr(OpExpr *clause);
extern Query *inline_set_returning_function(PlannerInfo *root,
RangeTblEntry *rte);
+extern bool is_parallel_allowed_for_modify(Query *parse);
#endif /* CLAUSES_H */
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
new file mode 100644
index 0000000000..d5fae79031
--- /dev/null
+++ b/src/test/regress/expected/insert_parallel.out
@@ -0,0 +1,536 @@
+--
+-- PARALLEL
+--
+--
+-- START: setup some tables and data needed by the tests.
+--
+-- Setup - index expressions test
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+-- Setup - column default tests
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+--
+-- END: setup some tables and data needed by the tests.
+--
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+ QUERY PLAN
+----------------------------------------------
+ Insert on para_insert_p1
+ -> Gather Merge
+ Workers Planned: 4
+ -> Sort
+ Sort Key: tenk1.unique1
+ -> Parallel Seq Scan on tenk1
+(6 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+ count
+-------
+ 1
+(1 row)
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_data1
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+ Filter: (a = 10)
+(5 rows)
+
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+ data
+------
+ 10
+(1 row)
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on para_insert_f1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on test_conflict_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on test_conflict_table
+ Conflict Resolution: UPDATE
+ Conflict Arbiter Indexes: test_conflict_table_pkey
+ -> Seq Scan on test_data
+(4 rows)
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names2
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names4
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+ QUERY PLAN
+----------------------------------------
+ Insert on names5
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names6
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names6 select * from names order by last_name returning *;
+ index | first_name | last_name
+-------+------------+-------------
+ 2 | niels | bohr
+ 1 | albert | einstein
+ 4 | leonhard | euler
+ 8 | richard | feynman
+ 5 | stephen | hawking
+ 6 | isaac | newton
+ 3 | erwin | schrodinger
+ 7 | alan | turing
+(8 rows)
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ QUERY PLAN
+----------------------------------------------
+ Insert on names7
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: names.last_name
+ -> Parallel Seq Scan on names
+(6 rows)
+
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+ last_name_then_first_name
+---------------------------
+ bohr, niels
+ einstein, albert
+ euler, leonhard
+ feynman, richard
+ hawking, stephen
+ newton, isaac
+ schrodinger, erwin
+ turing, alan
+(8 rows)
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on temp_names
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into temp_names select * from names;
+--
+-- Test INSERT with column defaults
+--
+--
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 10 | 8
+ 2 | 4 | 10 | 16
+ 3 | 6 | 10 | 24
+ 4 | 8 | 10 | 32
+ 5 | 10 | 10 | 40
+ 6 | 12 | 10 | 48
+ 7 | 14 | 10 | 56
+ 8 | 16 | 10 | 64
+ 9 | 18 | 10 | 72
+ 10 | 20 | 10 | 80
+(10 rows)
+
+truncate testdef;
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+ QUERY PLAN
+-----------------------------
+ Insert on testdef
+ -> Seq Scan on test_data
+(2 rows)
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on parttable1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+ count
+-------
+ 5000
+(1 row)
+
+select count(*) from parttable1_2;
+ count
+-------
+ 5000
+(1 row)
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on table_check_b
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names_with_safe_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_safe
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+ QUERY PLAN
+-------------------------------------
+ Insert on names_with_unsafe_trigger
+ -> Seq Scan on names
+(2 rows)
+
+insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_after_trigger_unsafe
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------
+ Insert on part_unsafe_trigger
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+--------------------------------------
+ Insert on rp
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(5 rows)
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+ QUERY PLAN
+-------------------------------
+ Insert on rp
+ -> Seq Scan on foo
+ Filter: ((a % 2) = 0)
+(3 rows)
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+create table dom_table_u (x inotnull_u, y int);
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on dom_table_u
+ -> Seq Scan on tenk1
+(2 rows)
+
+rollback;
+--
+-- Clean up anything not created in the transaction
+--
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
diff --git a/src/test/regress/parallel_schedule b/src/test/regress/parallel_schedule
index c77b0d7342..e280198b17 100644
--- a/src/test/regress/parallel_schedule
+++ b/src/test/regress/parallel_schedule
@@ -90,6 +90,7 @@ test: rules psql psql_crosstab amutils stats_ext collate.linux.utf8
# run by itself so it can run parallel workers
test: select_parallel
test: write_parallel
+test: insert_parallel
# no relation related tests can be put in this group
test: publication subscription
diff --git a/src/test/regress/serial_schedule b/src/test/regress/serial_schedule
index 0264a97324..6a57e889a1 100644
--- a/src/test/regress/serial_schedule
+++ b/src/test/regress/serial_schedule
@@ -148,6 +148,7 @@ test: stats_ext
test: collate.linux.utf8
test: select_parallel
test: write_parallel
+test: insert_parallel
test: publication
test: subscription
test: select_views
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
new file mode 100644
index 0000000000..70ad31a087
--- /dev/null
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -0,0 +1,335 @@
+--
+-- PARALLEL
+--
+
+--
+-- START: setup some tables and data needed by the tests.
+--
+
+-- Setup - index expressions test
+
+-- For testing purposes, we'll mark this function as parallel-unsafe
+create or replace function fullname_parallel_unsafe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel restricted;
+
+create table names(index int, first_name text, last_name text);
+create table names2(index int, first_name text, last_name text);
+create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names4(index int, first_name text, last_name text);
+create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
+
+insert into names values
+ (1, 'albert', 'einstein'),
+ (2, 'niels', 'bohr'),
+ (3, 'erwin', 'schrodinger'),
+ (4, 'leonhard', 'euler'),
+ (5, 'stephen', 'hawking'),
+ (6, 'isaac', 'newton'),
+ (7, 'alan', 'turing'),
+ (8, 'richard', 'feynman');
+
+-- Setup - column default tests
+
+create or replace function bdefault_unsafe ()
+returns int language plpgsql parallel unsafe as $$
+begin
+ RETURN 5;
+end $$;
+
+create or replace function cdefault_restricted ()
+returns int language plpgsql parallel restricted as $$
+begin
+ RETURN 10;
+end $$;
+
+create or replace function ddefault_safe ()
+returns int language plpgsql parallel safe as $$
+begin
+ RETURN 20;
+end $$;
+
+create table testdef(a int, b int default bdefault_unsafe(), c int default cdefault_restricted(), d int default ddefault_safe());
+
+create table test_data(a int);
+insert into test_data select * from generate_series(1,10);
+
+--
+-- END: setup some tables and data needed by the tests.
+--
+
+-- Serializable isolation would disable parallel query, so explicitly use an
+-- arbitrary other level.
+begin isolation level repeatable read;
+
+-- encourage use of parallel plans
+set parallel_setup_cost=0;
+set parallel_tuple_cost=0;
+set min_parallel_table_scan_size=0;
+set max_parallel_workers_per_gather=4;
+
+create table para_insert_p1 (
+ unique1 int4 PRIMARY KEY,
+ stringu1 name
+);
+
+create table para_insert_f1 (
+ unique1 int4 REFERENCES para_insert_p1(unique1),
+ stringu1 name
+);
+
+
+--
+-- Test INSERT with underlying query.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with ordered underlying query.
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+-- verify that the same transaction has been used by all parallel workers
+select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
+
+--
+-- Test INSERT with RETURNING clause.
+-- (should create plan with parallel SELECT, Gather parent node)
+--
+create table test_data1(like test_data);
+explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
+insert into test_data1 select * from test_data where a = 10 returning a as data;
+
+--
+-- Test INSERT into a table with a foreign key.
+-- (Insert into a table with a foreign key is parallel-restricted,
+-- as doing this in a parallel worker would create a new commandId
+-- and within a worker this is not currently supported)
+--
+explain (costs off) insert into para_insert_f1 select unique1, stringu1 from tenk1;
+insert into para_insert_f1 select unique1, stringu1 from tenk1;
+-- select some values to verify that the insert worked
+select count(*), sum(unique1) from para_insert_f1;
+
+--
+-- Test INSERT with ON CONFLICT ... DO UPDATE ...
+-- (should not create a parallel plan)
+--
+create table test_conflict_table(id serial primary key, somedata int);
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
+insert into test_conflict_table(id, somedata) select a, a from test_data;
+explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+
+
+--
+-- Test INSERT with parallel-unsafe index expression
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into names2 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names4 select * from names;
+
+--
+-- Test INSERT with underlying query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names5 (like names);
+explain (costs off) insert into names5 select * from names returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (no projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names6 (like names);
+explain (costs off) insert into names6 select * from names order by last_name returning *;
+insert into names6 select * from names order by last_name returning *;
+
+--
+-- Test INSERT with underlying ordered query - and RETURNING (with projection)
+-- (should create a parallel plan; parallel SELECT)
+--
+create table names7 (like names);
+explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+
+
+--
+-- Test INSERT into temporary table with underlying query.
+-- (Insert into a temp table is parallel-restricted;
+-- should create a parallel plan; parallel SELECT)
+--
+create temporary table temp_names (like names);
+explain (costs off) insert into temp_names select * from names;
+insert into temp_names select * from names;
+
+--
+-- Test INSERT with column defaults
+--
+--
+
+--
+-- Parallel unsafe column default, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
+
+--
+-- Parallel restricted column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
+--
+-- Parallel restricted and unsafe column defaults, should not use a parallel plan
+--
+explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
+
+--
+-- Test INSERT into partition with underlying query.
+--
+create table parttable1 (a int, b name) partition by range (a);
+create table parttable1_1 partition of parttable1 for values from (0) to (5000);
+create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+insert into parttable1 select unique1,stringu1 from tenk1;
+select count(*) from parttable1_1;
+select count(*) from parttable1_2;
+
+--
+-- Test INSERT into table with parallel-unsafe check constraint
+-- (should not create a parallel plan)
+--
+create or replace function check_b_unsafe(b name) returns boolean as $$
+ begin
+ return (b <> 'XXXXXX');
+ end;
+$$ language plpgsql parallel unsafe;
+
+create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
+explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe after stmt-level triggers
+-- (should create a parallel SELECT plan; triggers should fire)
+--
+create table names_with_safe_trigger (like names);
+create or replace function insert_after_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
+create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
+ for each statement execute procedure insert_after_trigger_safe();
+explain (costs off) insert into names_with_safe_trigger select * from names;
+insert into names_with_safe_trigger select * from names;
+
+--
+-- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- (should not create a parallel plan; triggers should fire)
+--
+create table names_with_unsafe_trigger (like names);
+create or replace function insert_after_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
+create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_after_trigger_unsafe();
+explain (costs off) insert into names_with_unsafe_trigger select * from names;
+insert into names_with_unsafe_trigger select * from names;
+
+--
+-- Test INSERT into partition with parallel-unsafe trigger
+-- (should not create a parallel plan)
+--
+
+create table part_unsafe_trigger (a int4, b name) partition by range (a);
+create table part_unsafe_trigger_1 partition of part_unsafe_trigger for values from (0) to (5000);
+create table part_unsafe_trigger_2 partition of part_unsafe_trigger for values from (5000) to (10000);
+create trigger part_insert_after_trigger_unsafe after insert on part_unsafe_trigger_1
+ for each statement execute procedure insert_after_trigger_unsafe();
+
+explain (costs off) insert into part_unsafe_trigger select unique1, stringu1 from tenk1;
+
+--
+-- Test that parallel-safety-related changes to partitions are detected and
+-- plan cache invalidation is working correctly.
+--
+
+create table rp (a int) partition by range (a);
+create table rp1 partition of rp for values from (minvalue) to (0);
+create table rp2 partition of rp for values from (0) to (maxvalue);
+create table foo (a) as select unique1 from tenk1;
+prepare q as insert into rp select * from foo where a%2 = 0;
+-- should create a parallel plan
+explain (costs off) execute q;
+
+create or replace function make_table_bar () returns trigger language
+plpgsql as $$ begin create table bar(); return null; end; $$ parallel unsafe;
+create trigger ai_rp2 after insert on rp2 for each row execute
+function make_table_bar();
+-- should create a non-parallel plan
+explain (costs off) execute q;
+
+--
+-- Test INSERT into table having a DOMAIN column with a CHECK constraint
+--
+create function sql_is_distinct_from_u(anyelement, anyelement)
+returns boolean language sql parallel unsafe
+as 'select $1 is distinct from $2 limit 1';
+
+create domain inotnull_u int
+ check (sql_is_distinct_from_u(value, null));
+
+create table dom_table_u (x inotnull_u, y int);
+
+
+-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
+explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+
+
+rollback;
+
+--
+-- Clean up anything not created in the transaction
+--
+
+drop table names;
+drop index names2_fullname_idx;
+drop table names2;
+drop index names4_fullname_idx;
+drop table names4;
+drop table testdef;
+drop table test_data;
+
+drop function bdefault_unsafe;
+drop function cdefault_restricted;
+drop function ddefault_safe;
+drop function fullname_parallel_unsafe;
+drop function fullname_parallel_restricted;
--
2.27.0
v25-0002-Add-new-parallel-insert-GUC-and-table-options.patchapplication/octet-stream; name=v25-0002-Add-new-parallel-insert-GUC-and-table-options.patchDownload
From 704823b0f4e87286d984d22d8d8fba0ac0016acb Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:17:34 +0530
Subject: [PATCH v25 2/3] Add new GUC option "enable_parallel_insert" (boolean)
and table option "parallel_insert_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_insert" is
added, to allow parallel insert to be enabled/disabled. The default is off.
In addition to the GUC option, the user may want a mechanism for specifying
parallel insert with finer granularity, to enable/disable the use of parallel
insert for specific tables.
The new table option "parallel_insert_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 ++++++++++
src/backend/access/common/reloptions.c | 25 ++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 ++++++++--
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 53 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 40 +++++++++++++-
14 files changed, 226 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73596..eed4037c9d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-insert" xreflabel="enable_parallel_insert">
+ <term><varname>enable_parallel_insert</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_insert</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ <command>INSERT</command> commands. The default is <literal>off</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for <command>INSERT</command>. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ <command>INSERT</command>.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..99a56f856c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_insert_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..00aa243b7f 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-insert-enabled" xreflabel="parallel_insert_enabled">
+ <term><literal>parallel_insert_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_insert_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel insert for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for <command>INSERT</command>. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel insert for this table.
+ Note that if the target table of the parallel insert is partitioned, the
+ <literal>parallel_insert_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..b582a7b87d 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_insert_enabled",
+ "Enables \"parallel insert\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_insert_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_insert_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..bf4bba8b05 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_insert = false;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index c416263749..46c6e65c49 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1272,8 +1272,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_insert is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_insert_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1284,12 +1286,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_insert)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1320,7 +1327,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if(!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_insert_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_insert_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelInsert(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3fd1a5fbe2..c67e363bbf 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_insert", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for INSERT commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_insert,
+ false,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..8b7a7856b7 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_insert = off
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..2ac98bb10f 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_insert_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..22e6db96b6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_insert;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..982e3db1a8 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelInsert
+ * Returns the relation's parallel_insert_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelInsert(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_insert_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_insert_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79031..f8d6dd8b4b 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,42 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +394,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..cc0dfa7355 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -107,13 +107,14 @@ select name, setting from pg_settings where name like 'enable%';
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
+ enable_parallel_insert | off
enable_partition_pruning | on
enable_partitionwise_aggregate | off
enable_partitionwise_join | off
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a087..70333a90b3 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,34 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +231,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.27.0
v25-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v25-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From cef775a1fed73c884a676057aa527da0f1ae0ce9 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH v25 3/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index f8d6dd8b4b..c070aa8afa 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -179,6 +201,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -202,6 +251,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -226,6 +355,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -322,6 +653,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -380,6 +790,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -438,6 +877,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -456,16 +953,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -478,18 +983,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -500,8 +1014,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -550,15 +1099,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -567,6 +1155,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -574,6 +1197,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -582,4 +1207,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70333a90b3..6577d5593d 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -139,6 +165,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -150,6 +184,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -223,6 +382,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -255,6 +422,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -269,37 +476,78 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -333,6 +581,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -340,15 +599,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -359,6 +651,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -368,4 +662,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
On Mon, Mar 8, 2021 at 3:54 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I've attached an updated set of patches with the suggested locking changes.
Amit L, others, do let me know if you have still more comments on
0001* patch or if you want to review it further?
--
With Regards,
Amit Kapila.
Hi Amit
On Mon, Mar 8, 2021 at 10:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 8, 2021 at 3:54 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I've attached an updated set of patches with the suggested locking changes.
(Thanks Greg.)
Amit L, others, do let me know if you have still more comments on
0001* patch or if you want to review it further?
I just read through v25 and didn't find anything to complain about.
--
Amit Langote
EDB: http://www.enterprisedb.com
I've attached an updated set of patches with the suggested locking changes.
Amit L, others, do let me know if you have still more comments on
0001* patch or if you want to review it further?
I took a look into the latest 0001 patch, and it looks good to me.
Best regards,
houzj
On Mon, Mar 8, 2021 at 7:19 PM Amit Langote <amitlangote09@gmail.com> wrote:
Hi Amit
On Mon, Mar 8, 2021 at 10:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 8, 2021 at 3:54 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I've attached an updated set of patches with the suggested locking changes.
(Thanks Greg.)
Amit L, others, do let me know if you have still more comments on
0001* patch or if you want to review it further?I just read through v25 and didn't find anything to complain about.
Thanks a lot, pushed now! Amit L., your inputs are valuable for this work.
Now, coming back to Hou-San's patch to introduce a GUC and reloption
for this feature, I think both of those make sense to me because when
the feature is enabled via GUC, one might want to disable it for
partitioned tables? Do we agree on that part or someone thinks
otherwise?
The other points to bikeshed could be:
1. The name of GUC and reloption. The two proposals at hand are
enable_parallel_dml and enable_parallel_insert. I would prefer the
second (enable_parallel_insert) because updates/deletes might not have
a similar overhead.
2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.
Thoughts?
--
With Regards,
Amit Kapila.
On Wed, Mar 10, 2021 at 2:48 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 8, 2021 at 7:19 PM Amit Langote <amitlangote09@gmail.com> wrote:
Hi Amit
On Mon, Mar 8, 2021 at 10:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 8, 2021 at 3:54 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
I've attached an updated set of patches with the suggested locking changes.
(Thanks Greg.)
Amit L, others, do let me know if you have still more comments on
0001* patch or if you want to review it further?I just read through v25 and didn't find anything to complain about.
Thanks a lot, pushed now! Amit L., your inputs are valuable for this work.
Now, coming back to Hou-San's patch to introduce a GUC and reloption
for this feature, I think both of those make sense to me because when
the feature is enabled via GUC, one might want to disable it for
partitioned tables? Do we agree on that part or someone thinks
otherwise?
I think it makes sense to have both.
The other points to bikeshed could be:
1. The name of GUC and reloption. The two proposals at hand are
enable_parallel_dml and enable_parallel_insert. I would prefer the
second (enable_parallel_insert) because updates/deletes might not have
a similar overhead.
I agree enable_parallel_insert makes more sense.
2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.Thoughts?
IMHO, we should keep it on because in most of the cases it is going
the give benefit to the user, and if for some specific scenario where
a table has a lot of partition then it can be turned off using
reloption. And, if for some users most of the tables are like that
where they always have a lot of partition then the user can turn it
off using guc.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.Thoughts?
IMHO, we should keep it on because in most of the cases it is going the give
benefit to the user, and if for some specific scenario where a table has a lot of
partition then it can be turned off using reloption. And, if for some users most
of the tables are like that where they always have a lot of partition then the user
can turn it off using guc.
I think your suggestion makes sense.
If no one have other opinions, I will post a new version set default enable_parallel_insert=on soon.
Best regards,
houzj
From: Amit Kapila <amit.kapila16@gmail.com>
Now, coming back to Hou-San's patch to introduce a GUC and reloption
for this feature, I think both of those make sense to me because when
the feature is enabled via GUC, one might want to disable it for
partitioned tables? Do we agree on that part or someone thinks
otherwise?The other points to bikeshed could be:
1. The name of GUC and reloption. The two proposals at hand are
enable_parallel_dml and enable_parallel_insert. I would prefer the
second (enable_parallel_insert) because updates/deletes might not have
a similar overhead.2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.Thoughts?
TBH, I'm a bit unsure, but the names with _insert would be OK.
The reason why Oracle has ENABLE PARALLEL DML clause and the parallel DML is disabled by default is probably due to the following critical restriction (and they have other less severe restrictions in parallel execution.) Our implementation does not have things like this.
"Serial or parallel statements that attempt to access a table that has been modified in parallel within the same transaction are rejected with an error message."
"A transaction can contain multiple parallel DML statements that modify different tables, but after a parallel DML statement modifies a table, no subsequent serial or parallel statement (DML or query) can access the same table again in that transaction."
OTOH, I'm a bit concerned about whether we would have a common reason to disable parallel INSERT, UPDATE and DELETE. Oracle states space usage difference as follows. I wonder if something similar would apply to our parallel UPDATE/DELETE, particularly UPDATE. At least, we already know parallel INSERT results in larger tables and indexes compared to serial execution.
"Parallel UPDATE uses the existing free space in the object, while direct-path INSERT gets new extents for the data."
"Space usage characteristics may be different in parallel than serial execution because multiple concurrent child transactions modify the object."
Even if that's the case, we can add _update parameters, although we feel sorry to cause users trouble to have to be aware of multiple parameters.
Or, when it has turned out that we need _update and/or _delete parameters, we can opt to rename _insert to _dml, and keep the _insert parameter as an old, hidden synonym.
Regards
Takayuki Tsunakawa
On Wed, Mar 10, 2021 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Mar 8, 2021 at 7:19 PM Amit Langote <amitlangote09@gmail.com> wrote:
I just read through v25 and didn't find anything to complain about.
Thanks a lot, pushed now! Amit L., your inputs are valuable for this work.
Glad I could be of help. Really appreciate that you credited me as
author for whatever small part I contributed.
Now, coming back to Hou-San's patch to introduce a GUC and reloption
for this feature, I think both of those make sense to me because when
the feature is enabled via GUC, one might want to disable it for
partitioned tables? Do we agree on that part or someone thinks
otherwise?
Agree to have both.
The other points to bikeshed could be:
1. The name of GUC and reloption. The two proposals at hand are
enable_parallel_dml and enable_parallel_insert. I would prefer the
second (enable_parallel_insert) because updates/deletes might not have
a similar overhead.
Sounds reasonable.
2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.
This makes sense too.
--
Amit Langote
EDB: http://www.enterprisedb.com
On Wed, Mar 10, 2021 at 8:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Now, coming back to Hou-San's patch to introduce a GUC and reloption
for this feature, I think both of those make sense to me because when
the feature is enabled via GUC, one might want to disable it for
partitioned tables? Do we agree on that part or someone thinks
otherwise?
Having both makes sense to me.
The other points to bikeshed could be:
1. The name of GUC and reloption. The two proposals at hand are
enable_parallel_dml and enable_parallel_insert. I would prefer the
second (enable_parallel_insert) because updates/deletes might not have
a similar overhead.
I guess to have the finer granularity we'd have to go with
enable_parallel_insert, which then would mean possibly having to later
add enable_parallel_update, should parallel update have similar
potential overhead in the parallel-safety checks (which to me, looks
like it could, and parallel delete may not ...)
It's a shame there is no "set" type for GUC options.
e.g.
enable_parallel_dml='insert,update'
Maybe that's going too far.
2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.
I'd agree to keeping it on by default (and it can be selectively
turned off for a particular table using the reloption, if needed).
Regards,
Greg Nancarrow
Fujitsu Australia
I guess to have the finer granularity we'd have to go with enable_parallel_insert,
which then would mean possibly having to later add enable_parallel_update,
should parallel update have similar potential overhead in the parallel-safety
checks (which to me, looks like it could, and parallel delete may not ...)It's a shame there is no "set" type for GUC options.
e.g.
enable_parallel_dml='insert,update'
Maybe that's going too far.2. Should we keep the default value of GUC to on or off? It is
currently off. I am fine keeping it off for this release and we can
always turn it on in the later releases if required. Having said that,
I see the value in keeping it on because in many cases Insert ...
Select will be used for large data and there we will see a benefit of
parallelism and users facing trouble (who have a very large number of
partitions with less data to query) can still disable the parallel
insert for that particular table. Also, the other benefit of keeping
it on till at least the beta period is that this functionality will
get tested and if we found reports of regression then we can turn it
off for this release as well.I'd agree to keeping it on by default (and it can be selectively turned off for a
particular table using the reloption, if needed).
Thanks, attaching new version patch keeping the default value of guc option to ON.
Best regards,
houzj
Attachments:
v26-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v26-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From cef775a1fed73c884a676057aa527da0f1ae0ce9 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH v25 3/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index f8d6dd8b4b..c070aa8afa 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -179,6 +201,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -202,6 +251,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -226,6 +355,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -322,6 +653,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -380,6 +790,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -438,6 +877,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -456,16 +953,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -478,18 +983,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -500,8 +1014,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -550,15 +1099,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -567,6 +1155,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -574,6 +1197,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -582,4 +1207,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70333a90b3..6577d5593d 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -139,6 +165,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -150,6 +184,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -223,6 +382,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -255,6 +422,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -269,37 +476,78 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -333,6 +581,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -340,15 +599,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -359,6 +651,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -368,4 +662,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v26-0002-Add-new-GUC-option-enable_parallel_insert-boolean.patchapplication/octet-stream; name=v26-0002-Add-new-GUC-option-enable_parallel_insert-boolean.patchDownload
From 35b6d0aff37a34aa41da7f80f19b78a200b61ddd Mon Sep 17 00:00:00 2001
From: "houzj.fnst" <houzj.fnst@cn.fujitsu.com>
Date: Thu, 11 Mar 2021 09:11:22 +0800
Subject: [PATCH] Add new GUC option enable_parallel_insert (boolean)
and table option "parallel_insert_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_insert" is
added, to allow parallel insert to be enabled/disabled. The default is on.
In addition to the GUC option, the user may want a mechanism for specifying
parallel insert with finer granularity, to enable/disable the use of parallel
insert for specific tables.
The new table option "parallel_insert_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 +++++++++
src/backend/access/common/reloptions.c | 25 +++++++--
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 29 +++++++++-
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 56 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 44 ++++++++++++++-
14 files changed, 233 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 967de73596..9264b5b542 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-insert" xreflabel="enable_parallel_insert">
+ <term><varname>enable_parallel_insert</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_insert</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ <command>INSERT</command> commands. The default is <literal>on</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for <command>INSERT</command>. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ <command>INSERT</command>.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..99a56f856c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_insert_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..00aa243b7f 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-insert-enabled" xreflabel="parallel_insert_enabled">
+ <term><literal>parallel_insert_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_insert_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel insert for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for <command>INSERT</command>. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel insert for this table.
+ Note that if the target table of the parallel insert is partitioned, the
+ <literal>parallel_insert_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index c687d3ee9e..b582a7b87d 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_insert_enabled",
+ "Enables \"parallel insert\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_insert_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedOptions, parallel_insert_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..c81e2cf244 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_insert = true;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index c416263749..46c6e65c49 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1272,8 +1272,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_insert is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_insert_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1284,12 +1286,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_insert)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1320,7 +1327,23 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if (!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_insert_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_insert_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelInsert(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3fd1a5fbe2..3e45cf627c 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2049,6 +2049,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_insert", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for INSERT commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_insert,
+ true,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index ee06528bb0..ac804f4877 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_insert = on
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index 9f0208ac49..2ac98bb10f 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_insert_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..22e6db96b6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_insert;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..982e3db1a8 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
+} PartitionedOptions;
+
+/*
+ * RelationGetParallelInsert
+ * Returns the relation's parallel_insert_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelInsert(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedOptions *) (relation)->rd_options)->parallel_insert_enabled :\
+ ((StdRdOptions *) (relation)->rd_options)->parallel_insert_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79031..cb89453a14 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,45 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +397,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..a62bf5dc92 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -107,13 +107,14 @@ select name, setting from pg_settings where name like 'enable%';
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
+ enable_parallel_insert | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
enable_partitionwise_join | off
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a087..5092990a5b 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,38 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +235,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.18.4
The buildfarm has revealed that this patch doesn't work under
CLOBBER_CACHE_ALWAYS:
https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=husky&dt=2021-03-10%2021%3A09%3A32
I initially thought that that was a problem with c3ffe34863, but after
reproducing it here I get this stack trace:
#0 target_rel_trigger_max_parallel_hazard (context=0x7fff9cbfc220,
trigdesc=0x7fdd7dfa9718) at clauses.c:971
#1 target_rel_max_parallel_hazard_recurse (command_type=CMD_INSERT,
context=0x7fff9cbfc220, rel=0x7fdd7df819e0) at clauses.c:929
#2 target_rel_max_parallel_hazard_recurse (rel=0x7fdd7df819e0,
command_type=<optimized out>, context=0x7fff9cbfc220) at clauses.c:893
#3 0x000000000075d26e in target_rel_max_parallel_hazard (
context=0x7fff9cbfc220) at clauses.c:884
#4 max_parallel_hazard_walker (node=node@entry=0x146e590,
context=context@entry=0x7fff9cbfc220) at clauses.c:831
#5 0x0000000000700812 in range_table_entry_walker (rte=0x146e590,
walker=0x75cf00 <max_parallel_hazard_walker>, context=0x7fff9cbfc220,
flags=16) at nodeFuncs.c:2467
#6 0x0000000000700927 in range_table_walker (rtable=0x11fdd70,
walker=walker@entry=0x75cf00 <max_parallel_hazard_walker>,
context=context@entry=0x7fff9cbfc220, flags=16) at nodeFuncs.c:2446
#7 0x0000000000700ada in query_tree_walker (flags=<optimized out>,
context=0x7fff9cbfc220, walker=0x75cf00 <max_parallel_hazard_walker>,
query=0x11fdc58) at nodeFuncs.c:2423
#8 query_tree_walker (query=query@entry=0x700927 <range_table_walker+87>,
walker=walker@entry=0x75cf00 <max_parallel_hazard_walker>,
context=context@entry=0x11fdc58, flags=<optimized out>) at nodeFuncs.c:2336
#9 0x000000000075d138 in max_parallel_hazard_walker (
node=node@entry=0x11fdc58, context=0x11fdc58, context@entry=0x7fff9cbfc220)
at clauses.c:853
#10 0x000000000075dc98 in max_parallel_hazard (parse=parse@entry=0x11fdc58,
glob=glob@entry=0x11fdb40) at clauses.c:585
#11 0x000000000074cd22 in standard_planner (parse=0x11fdc58,
query_string=<optimized out>, cursorOptions=256,
boundParams=<optimized out>) at planner.c:345
#12 0x0000000000814947 in pg_plan_query (querytree=0x11fdc58,
query_string=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;", cursorOptions=256, boundParams=0x0)
at postgres.c:809
#13 0x0000000000814a43 in pg_plan_queries (querytrees=0x14725d0,
query_string=query_string@entry=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;",
cursorOptions=cursorOptions@entry=256, boundParams=boundParams@entry=0x0)
at postgres.c:900
#14 0x0000000000814d35 in exec_simple_query (
query_string=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;") at postgres.c:1092
The problem is that target_rel_trigger_max_parallel_hazard and its caller
think they can use a relcache TriggerDesc field across other cache
accesses, which they can't because the relcache doesn't guarantee that
that won't move.
One approach would be to add logic to RelationClearRelation similar to
what it does for tupdescs, rules, etc, to avoid moving them when their
contents haven't changed. But given that we've not needed that for the
past several decades, I'm disinclined to add the overhead. I think this
code ought to be adjusted to not make its own copy of the trigdesc
pointer, but instead fetch it out of the relcache struct each time it is
accessed. There's no real reason why
target_rel_trigger_max_parallel_hazard shouldn't be passed the (stable)
Relation pointer instead of just the trigdesc pointer.
BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values
for those trigger functions?
regards, tom lane
On Thu, Mar 11, 2021 at 01:01:42PM +0000, houzj.fnst@fujitsu.com wrote:
I guess to have the finer granularity we'd have to go with enable_parallel_insert,
which then would mean possibly having to later add enable_parallel_update,
should parallel update have similar potential overhead in the parallel-safety
checks (which to me, looks like it could, and parallel delete may not ...)It's a shame there is no "set" type for GUC options.
e.g.
enable_parallel_dml='insert,update'
Maybe that's going too far.
Isn't that just GUC_LIST_INPUT ?
I'm not sure why it'd be going to far ?
The GUC-setting assign hook can parse the enable_parallel_dml_list value set by
the user, and set an internal int/bits enable_parallel_dml variable with some
define/enum values like:
GUC_PARALLEL_DML_INSERT 0x01
GUC_PARALLEL_DML_DELETE 0x02
GUC_PARALLEL_DML_UPDATE 0x04
The namespace.c assign hook is a good prototype for this. The parsed, integer
GUC can probably be a static variable in clauses.c.
Then, the planner can check if:
|commandType == CMD_INSERT &&
| (enable_parallel_dml & GUC_PARALLEL_DML_INSERT) != 0
[...]
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
It seems like this usefully allows the GUC to be enabled, and reloption to be
disabled. But if the GUC is disabled, then it's impossible to enable for a
single table. That seems unfortunate. I think part of the issue is the
naming. If the GUC is called "enable_*", then setting it to "off" should
disable it entirely, for consistency with other GUCs. So maybe it needs
another name, like parallel_dml='insert'. I think maybe "all" should be
an accepted value.
Note also this CF entry
https://commitfest.postgresql.org/32/2987/
| Allow setting parallel_workers on partitioned tables
--
Justin
On Fri, Mar 12, 2021 at 5:00 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
The buildfarm has revealed that this patch doesn't work under
CLOBBER_CACHE_ALWAYS:https://buildfarm.postgresql.org/cgi-bin/show_log.pl?nm=husky&dt=2021-03-10%2021%3A09%3A32
I initially thought that that was a problem with c3ffe34863, but after
reproducing it here I get this stack trace:#0 target_rel_trigger_max_parallel_hazard (context=0x7fff9cbfc220,
trigdesc=0x7fdd7dfa9718) at clauses.c:971
#1 target_rel_max_parallel_hazard_recurse (command_type=CMD_INSERT,
context=0x7fff9cbfc220, rel=0x7fdd7df819e0) at clauses.c:929
#2 target_rel_max_parallel_hazard_recurse (rel=0x7fdd7df819e0,
command_type=<optimized out>, context=0x7fff9cbfc220) at clauses.c:893
#3 0x000000000075d26e in target_rel_max_parallel_hazard (
context=0x7fff9cbfc220) at clauses.c:884
#4 max_parallel_hazard_walker (node=node@entry=0x146e590,
context=context@entry=0x7fff9cbfc220) at clauses.c:831
#5 0x0000000000700812 in range_table_entry_walker (rte=0x146e590,
walker=0x75cf00 <max_parallel_hazard_walker>, context=0x7fff9cbfc220,
flags=16) at nodeFuncs.c:2467
#6 0x0000000000700927 in range_table_walker (rtable=0x11fdd70,
walker=walker@entry=0x75cf00 <max_parallel_hazard_walker>,
context=context@entry=0x7fff9cbfc220, flags=16) at nodeFuncs.c:2446
#7 0x0000000000700ada in query_tree_walker (flags=<optimized out>,
context=0x7fff9cbfc220, walker=0x75cf00 <max_parallel_hazard_walker>,
query=0x11fdc58) at nodeFuncs.c:2423
#8 query_tree_walker (query=query@entry=0x700927 <range_table_walker+87>,
walker=walker@entry=0x75cf00 <max_parallel_hazard_walker>,
context=context@entry=0x11fdc58, flags=<optimized out>) at nodeFuncs.c:2336
#9 0x000000000075d138 in max_parallel_hazard_walker (
node=node@entry=0x11fdc58, context=0x11fdc58, context@entry=0x7fff9cbfc220)
at clauses.c:853
#10 0x000000000075dc98 in max_parallel_hazard (parse=parse@entry=0x11fdc58,
glob=glob@entry=0x11fdb40) at clauses.c:585
#11 0x000000000074cd22 in standard_planner (parse=0x11fdc58,
query_string=<optimized out>, cursorOptions=256,
boundParams=<optimized out>) at planner.c:345
#12 0x0000000000814947 in pg_plan_query (querytree=0x11fdc58,
query_string=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;", cursorOptions=256, boundParams=0x0)
at postgres.c:809
#13 0x0000000000814a43 in pg_plan_queries (querytrees=0x14725d0,
query_string=query_string@entry=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;",
cursorOptions=cursorOptions@entry=256, boundParams=boundParams@entry=0x0)
at postgres.c:900
#14 0x0000000000814d35 in exec_simple_query (
query_string=0x11fc740 "insert into fk_notpartitioned_pk (a, b)\n select 2048, x from generate_series(1,10) x;") at postgres.c:1092The problem is that target_rel_trigger_max_parallel_hazard and its caller
think they can use a relcache TriggerDesc field across other cache
accesses, which they can't because the relcache doesn't guarantee that
that won't move.One approach would be to add logic to RelationClearRelation similar to
what it does for tupdescs, rules, etc, to avoid moving them when their
contents haven't changed. But given that we've not needed that for the
past several decades, I'm disinclined to add the overhead. I think this
code ought to be adjusted to not make its own copy of the trigdesc
pointer, but instead fetch it out of the relcache struct each time it is
accessed. There's no real reason why
target_rel_trigger_max_parallel_hazard shouldn't be passed the (stable)
Relation pointer instead of just the trigdesc pointer.BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values
for those trigger functions?
Thanks Tom for your investigation, detailed analysis and suggested code fixes.
Will work on getting these issues corrected ASAP (and, er, removing
the looniness ...).
Regards,
Greg Nancarrow
Fujitsu Australia
On Thu, Mar 11, 2021 at 01:01:42PM +0000, houzj.fnst@fujitsu.com wrote:
I guess to have the finer granularity we'd have to go with
enable_parallel_insert, which then would mean possibly having to
later add enable_parallel_update, should parallel update have
similar potential overhead in the parallel-safety checks (which to
me, looks like it could, and parallel delete may not ...)It's a shame there is no "set" type for GUC options.
e.g.
enable_parallel_dml='insert,update'
Maybe that's going too far.Isn't that just GUC_LIST_INPUT ?
I'm not sure why it'd be going to far ?The GUC-setting assign hook can parse the enable_parallel_dml_list value set
by the user, and set an internal int/bits enable_parallel_dml variable with some
define/enum values like:GUC_PARALLEL_DML_INSERT 0x01
GUC_PARALLEL_DML_DELETE 0x02
GUC_PARALLEL_DML_UPDATE 0x04The namespace.c assign hook is a good prototype for this. The parsed,
integer GUC can probably be a static variable in clauses.c.Then, the planner can check if:
|commandType == CMD_INSERT &&
| (enable_parallel_dml & GUC_PARALLEL_DML_INSERT) != 0
[...]+ this table. When enabled (and provided that + <xref linkend="guc-enable-parallel-insert"/> is also + <literal>true</literal>),
I think this ideas works, but we still need to consider about the reloption.
After looking into the reloption, I think postgres do not have a list-like type for reloption.
And I think it's better that the guc and reloption is consistent.
Besides, a list type guc option that only support one valid value 'insert' seems a little weird to me(we only support parallel insert for now).
So, I tend to keep the current style of guc option.
If we turn out that we do need same option to restrict update/delete, we can improve this in the future
What do you think ?
Best regards,
houzj
On Fri, Mar 12, 2021 at 6:10 AM Justin Pryzby <pryzby@telsasoft.com> wrote:
Note also this CF entry
https://commitfest.postgresql.org/32/2987/
| Allow setting parallel_workers on partitioned tables
+/*
+ * PartitionedOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use
of parallel insert */
+} PartitionedOptions;
houzj, could you please consider naming the struct
PartitionedTableRdOptions as the patch for adding parallel_workers
option to partitioned tables does?
--
Amit Langote
EDB: http://www.enterprisedb.com
On Fri, Mar 12, 2021 at 5:00 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
The problem is that target_rel_trigger_max_parallel_hazard and its caller
think they can use a relcache TriggerDesc field across other cache
accesses, which they can't because the relcache doesn't guarantee that
that won't move.One approach would be to add logic to RelationClearRelation similar to
what it does for tupdescs, rules, etc, to avoid moving them when their
contents haven't changed. But given that we've not needed that for the
past several decades, I'm disinclined to add the overhead. I think this
code ought to be adjusted to not make its own copy of the trigdesc
pointer, but instead fetch it out of the relcache struct each time it is
accessed. There's no real reason why
target_rel_trigger_max_parallel_hazard shouldn't be passed the (stable)
Relation pointer instead of just the trigdesc pointer.
I have attached a patch to fix the issue, based on your suggestion
(tested with CLOBBER_CACHE_ALWAYS defined).
BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values
for those trigger functions?
... and also attached a patch to update the code for this issue.
(2nd patch relies on application of the 1st patch)
Thanks again for pointing out these problems.
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v1-0001-Fix-TriggerDesc-relcache-bug-introduced-by-recent-commit.patchapplication/octet-stream; name=v1-0001-Fix-TriggerDesc-relcache-bug-introduced-by-recent-commit.patchDownload
From 5625506fb597c61ef361a6a64f90eeea0eb12797 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 12 Mar 2021 17:16:19 +1100
Subject: [PATCH 1/1] Fix TriggerDesc relcache bug introduced by recent commit
05c8482.
The commit added code which used a relcache TriggerDesc field across another
cache access, which it shouldn't because the relcache doesn't guarantee it
won't get moved.
Discussion: https://www.postgresql.org/message-id/flat/CA%2BHiwqHJ3sp9hAZF1vM3F55%3D8kuDnMkESrTACN5A4TMw6JznUg%40mail.gmail.com
---
src/backend/optimizer/util/clauses.c | 24 +++++++++++++++---------
1 file changed, 15 insertions(+), 9 deletions(-)
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 7e25f94293..12754fdb2b 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -114,7 +114,7 @@ static bool target_rel_max_parallel_hazard(max_parallel_hazard_context *context)
static bool target_rel_max_parallel_hazard_recurse(Relation relation,
CmdType command_type,
max_parallel_hazard_context *context);
-static bool target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+static bool target_rel_trigger_max_parallel_hazard(Relation rel,
max_parallel_hazard_context *context);
static bool target_rel_index_max_parallel_hazard(Relation rel,
max_parallel_hazard_context *context);
@@ -926,7 +926,7 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
/*
* If any triggers exist, check that they are parallel-safe.
*/
- if (target_rel_trigger_max_parallel_hazard(rel->trigdesc, context))
+ if (target_rel_trigger_max_parallel_hazard(rel, context))
return true;
/*
@@ -952,23 +952,29 @@ target_rel_max_parallel_hazard_recurse(Relation rel,
/*
* target_rel_trigger_max_parallel_hazard
*
- * Finds the maximum parallel-mode hazard level for the specified trigger data.
+ * Finds the maximum parallel-mode hazard level for the specified relation's
+ * trigger data.
*/
static bool
-target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
+target_rel_trigger_max_parallel_hazard(Relation rel,
max_parallel_hazard_context *context)
{
int i;
- if (trigdesc == NULL)
+ if (rel->trigdesc == NULL)
return false;
- for (i = 0; i < trigdesc->numtriggers; i++)
+ /*
+ * Care is needed here to avoid using the same relcache TriggerDesc field
+ * across other cache accesses, because relcache doesn't guarantee that it
+ * won't move.
+ */
+ for (i = 0; i < rel->trigdesc->numtriggers; i++)
{
int trigtype;
- Trigger *trigger = &trigdesc->triggers[i];
+ Oid tgfoid = rel->trigdesc->triggers[i].tgfoid;
- if (max_parallel_hazard_test(func_parallel(trigger->tgfoid), context))
+ if (max_parallel_hazard_test(func_parallel(tgfoid), context))
return true;
/*
@@ -977,7 +983,7 @@ target_rel_trigger_max_parallel_hazard(TriggerDesc *trigdesc,
* on insert/update and this isn't supported in a parallel worker (but
* is safe in the parallel leader).
*/
- trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ trigtype = RI_FKey_trigger_type(tgfoid);
if (trigtype == RI_TRIGGER_FK)
{
if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
--
2.27.0
v1-0001-Better-implement-FK-trigger-parallel-safety-checking.patchapplication/octet-stream; name=v1-0001-Better-implement-FK-trigger-parallel-safety-checking.patchDownload
From de1a8e02b051359fe59cd03b9870656b973bc643 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Fri, 12 Mar 2021 17:25:21 +1100
Subject: [PATCH 1/1] Better implement FK trigger parallel-safety checking
added by commit 05c8482.
Commit 05c8482 added special logic related to parallel-safety of FK triggers.
This is a bit of a hack and should have instead been done by simply setting
appropriate proparallel values on those trigger functions themselves.
Discussion: https://www.postgresql.org/message-id/flat/CA%2BHiwqHJ3sp9hAZF1vM3F55%3D8kuDnMkESrTACN5A4TMw6JznUg%40mail.gmail.com
---
src/backend/optimizer/util/clauses.c | 13 -------------
src/include/catalog/catversion.h | 2 +-
src/include/catalog/pg_proc.dat | 8 ++++----
3 files changed, 5 insertions(+), 18 deletions(-)
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 12754fdb2b..74cc8ddda7 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -976,19 +976,6 @@ target_rel_trigger_max_parallel_hazard(Relation rel,
if (max_parallel_hazard_test(func_parallel(tgfoid), context))
return true;
-
- /*
- * If the trigger type is RI_TRIGGER_FK, this indicates a FK exists in
- * the relation, and this would result in creation of new CommandIds
- * on insert/update and this isn't supported in a parallel worker (but
- * is safe in the parallel leader).
- */
- trigtype = RI_FKey_trigger_type(tgfoid);
- if (trigtype == RI_TRIGGER_FK)
- {
- if (max_parallel_hazard_test(PROPARALLEL_RESTRICTED, context))
- return true;
- }
}
return false;
diff --git a/src/include/catalog/catversion.h b/src/include/catalog/catversion.h
index 33bb6f9672..6c4d9f0689 100644
--- a/src/include/catalog/catversion.h
+++ b/src/include/catalog/catversion.h
@@ -53,6 +53,6 @@
*/
/* yyyymmddN */
-#define CATALOG_VERSION_NO 202103093
+#define CATALOG_VERSION_NO 202103121
#endif
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 61361a6bc9..93393fcfd4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -3745,11 +3745,11 @@
# Generic referential integrity constraint triggers
{ oid => '1644', descr => 'referential integrity FOREIGN KEY ... REFERENCES',
- proname => 'RI_FKey_check_ins', provolatile => 'v', prorettype => 'trigger',
- proargtypes => '', prosrc => 'RI_FKey_check_ins' },
+ proname => 'RI_FKey_check_ins', provolatile => 'v', proparallel => 'r',
+ prorettype => 'trigger', proargtypes => '', prosrc => 'RI_FKey_check_ins' },
{ oid => '1645', descr => 'referential integrity FOREIGN KEY ... REFERENCES',
- proname => 'RI_FKey_check_upd', provolatile => 'v', prorettype => 'trigger',
- proargtypes => '', prosrc => 'RI_FKey_check_upd' },
+ proname => 'RI_FKey_check_upd', provolatile => 'v', proparallel => 'r',
+ prorettype => 'trigger', proargtypes => '', prosrc => 'RI_FKey_check_upd' },
{ oid => '1646', descr => 'referential integrity ON DELETE CASCADE',
proname => 'RI_FKey_cascade_del', provolatile => 'v', prorettype => 'trigger',
proargtypes => '', prosrc => 'RI_FKey_cascade_del' },
--
2.27.0
The problem is that target_rel_trigger_max_parallel_hazard and its
caller think they can use a relcache TriggerDesc field across other
cache accesses, which they can't because the relcache doesn't
guarantee that that won't move.One approach would be to add logic to RelationClearRelation similar to
what it does for tupdescs, rules, etc, to avoid moving them when their
contents haven't changed. But given that we've not needed that for
the past several decades, I'm disinclined to add the overhead. I
think this code ought to be adjusted to not make its own copy of the
trigdesc pointer, but instead fetch it out of the relcache struct each
time it is accessed. There's no real reason why
target_rel_trigger_max_parallel_hazard shouldn't be passed the
(stable) Relation pointer instead of just the trigdesc pointer.I have attached a patch to fix the issue, based on your suggestion (tested with
CLOBBER_CACHE_ALWAYS defined).BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values for
those trigger functions?... and also attached a patch to update the code for this issue.
(2nd patch relies on application of the 1st patch)
Thanks again for pointing out these problems.
I have tested the triggerdesc bugfix patch with CLOBBER_CACHE_ALWAYS flag.
It passed the testset where is fail in buildfarm (foreign_key, foreign_data).
And the patch LGTM.
Best regards,
houzj
On Fri, Mar 12, 2021 at 6:10 AM Justin Pryzby <mailto:pryzby@telsasoft.com> wrote:
Note also this CF entry
https://commitfest.postgresql.org/32/2987/
| Allow setting parallel_workers on partitioned tables+/* + * PartitionedOptions + * Contents of rd_options for partitioned tables + */ +typedef struct PartitionedOptions +{ + int32 vl_len_; /* varlena header (do not touch directly!) */ + bool parallel_insert_enabled; /* enables planner's use of parallel insert */ +} PartitionedOptions;houzj, could you please consider naming the struct PartitionedTableRdOptions
as the patch for adding parallel_workers option to partitioned tables does?
Thanks for reminding.
I agreed that " PartitionedTableRdOptions " is better.
Attaching new version patch with this change.
Best regards,
houzj
Attachments:
v27-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v27-0003-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From cef775a1fed73c884a676057aa527da0f1ae0ce9 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH v25 3/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index f8d6dd8b4b..c070aa8afa 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -179,6 +201,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -202,6 +251,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -226,6 +355,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -322,6 +653,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -342,6 +718,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -380,6 +790,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -438,6 +877,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -456,16 +953,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -478,18 +983,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -500,8 +1014,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -550,15 +1099,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -567,6 +1155,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -574,6 +1197,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -582,4 +1207,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70333a90b3..6577d5593d 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -139,6 +165,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -150,6 +184,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -159,6 +216,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -195,6 +316,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -209,6 +355,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -223,6 +382,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -255,6 +422,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -269,37 +476,78 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -333,6 +581,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -340,15 +599,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -359,6 +651,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -368,4 +662,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v27-0002-Add-new-GUC-option-enable_parallel_insert-boolean.patchapplication/octet-stream; name=v27-0002-Add-new-GUC-option-enable_parallel_insert-boolean.patchDownload
From 5be6c14ef17b9fa2641fb5df47fb616b85a21e0c Mon Sep 17 00:00:00 2001
From: "houzj.fnst" <houzj.fnst@cn.fujitsu.com>
Date: Fri, 12 Mar 2021 16:20:10 +0800
Subject: [PATCH] Add new GUC option enable_parallel_insert (boolean)
and table option "parallel_insert_enabled" (boolean).
The current implementation of parallel SELECT for "INSERT INTO ... SELECT ..."
may incur non-negligible overhead in the additional parallel-safety checks that
it performs, even when, in the end, those checks determine that parallelism
can't be used. This is normally only ever a problem for large complex tables,
particularly in the case of when the target table has a large number of
partitions.
To address this potential isse, a new GUC option "enable_parallel_insert" is
added, to allow parallel insert to be enabled/disabled. The default is on.
In addition to the GUC option, the user may want a mechanism for specifying
parallel insert with finer granularity, to enable/disable the use of parallel
insert for specific tables.
The new table option "parallel_insert_enabled" allows this. The default is true.
Author: "Hou, Zhijie" <houzj.fnst@cn.fujitsu.com>
Discussion: https://www.postgresql.org/message-id/flat/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
---
doc/src/sgml/config.sgml | 23 ++++++++
doc/src/sgml/ref/alter_table.sgml | 2 +-
doc/src/sgml/ref/create_table.sgml | 27 +++++++++
src/backend/access/common/reloptions.c | 25 +++++++--
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 34 ++++++++++-
src/backend/utils/misc/guc.c | 11 ++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 23 ++++++++
src/test/regress/expected/insert_parallel.out | 56 ++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 44 ++++++++++++++-
14 files changed, 238 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a218d78bef..6fe962b81e 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-insert" xreflabel="enable_parallel_insert">
+ <term><varname>enable_parallel_insert</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_insert</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ <command>INSERT</command> commands. The default is <literal>on</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for <command>INSERT</command>. In cases
+ such as when the target table has a large number of partitions, and
+ particularly also when that table uses something parallel-unsafe that
+ prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ <command>INSERT</command>.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5abd6..99a56f856c 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,7 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and <varname>parallel_insert_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 3b2b227683..00aa243b7f 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1408,6 +1408,33 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-insert-enabled" xreflabel="parallel_insert_enabled">
+ <term><literal>parallel_insert_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_insert_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel insert for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for <command>INSERT</command>. The default is
+ <literal>true</literal>.
+ In cases such as when the table has a large number of partitions, and
+ particularly also when that table uses a parallel-unsafe feature that
+ prevents parallelism, the overhead of these checks may become prohibitively
+ high. To address this potential overhead in these cases, this option can be
+ used to disable the use of parallel insert for this table.
+ Note that if the target table of the parallel insert is partitioned, the
+ <literal>parallel_insert_enabled</literal> option values of the partitions are
+ ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index d897bbec2b..5a0ae99750 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_insert_enabled",
+ "Enables \"parallel insert\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_insert_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedTableRdOptions, parallel_insert_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedTableRdOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674a19..c81e2cf244 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_insert = true;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index 7e25f94293..8ee92ee491 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1273,8 +1273,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_insert is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_insert_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1285,12 +1287,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_insert)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1321,7 +1328,28 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if (!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_insert_enabled is enabled for the target table,
+ * if not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_insert_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelInsert(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 855076b1fd..5460adc0d3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -2058,6 +2058,17 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
+ {
+ {"enable_parallel_insert", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for INSERT commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_insert,
+ true,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, false, NULL, NULL, NULL
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index f46c2dd7a8..6647f8fd6e 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_insert = on
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ecdb8d752b..d3fb734f05 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_insert_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be098..22e6db96b6 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_insert;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b63982c0..6a90716c0f 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,7 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -423,6 +424,28 @@ typedef struct ViewOptions
((ViewOptions *) (relation)->rd_options)->check_option == \
VIEW_OPTION_CHECK_OPTION_CASCADED)
+/*
+ * PartitionedTableRdOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedTableRdOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use of parallel insert */
+} PartitionedTableRdOptions;
+
+/*
+ * RelationGetParallelInsert
+ * Returns the relation's parallel_insert_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelInsert(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedTableRdOptions *) (relation)->rd_options)->parallel_insert_enabled : \
+ ((StdRdOptions *) (relation)->rd_options)->parallel_insert_enabled) : \
+ (defaultpd))
+
/*
* RelationIsValid
* True iff relation descriptor is valid.
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79031..cb89453a14 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,45 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +397,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e309c..a62bf5dc92 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -107,13 +107,14 @@ select name, setting from pg_settings where name like 'enable%';
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
+ enable_parallel_insert | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
enable_partitionwise_join | off
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a087..5092990a5b 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,38 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +235,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
--
2.18.4
On Fri, Mar 12, 2021 at 1:33 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
The problem is that target_rel_trigger_max_parallel_hazard and its
caller think they can use a relcache TriggerDesc field across other
cache accesses, which they can't because the relcache doesn't
guarantee that that won't move.One approach would be to add logic to RelationClearRelation similar to
what it does for tupdescs, rules, etc, to avoid moving them when their
contents haven't changed. But given that we've not needed that for
the past several decades, I'm disinclined to add the overhead. I
think this code ought to be adjusted to not make its own copy of the
trigdesc pointer, but instead fetch it out of the relcache struct each
time it is accessed. There's no real reason why
target_rel_trigger_max_parallel_hazard shouldn't be passed the
(stable) Relation pointer instead of just the trigdesc pointer.I have attached a patch to fix the issue, based on your suggestion (tested with
CLOBBER_CACHE_ALWAYS defined).BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values for
those trigger functions?... and also attached a patch to update the code for this issue.
(2nd patch relies on application of the 1st patch)
Thanks again for pointing out these problems.
I have tested the triggerdesc bugfix patch with CLOBBER_CACHE_ALWAYS flag.
It passed the testset where is fail in buildfarm (foreign_key, foreign_data).
Thanks for the patch and review. It looks good to me as well and
passes the tests (foreign_key, foreign_data) with CLOBBER_CACHE_ALWAYS
flag.
I'll review the second patch of Greg.
--
With Regards,
Amit Kapila.
On Fri, Mar 12, 2021 at 9:33 AM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
On Thu, Mar 11, 2021 at 01:01:42PM +0000, houzj.fnst@fujitsu.com wrote:
I guess to have the finer granularity we'd have to go with
enable_parallel_insert, which then would mean possibly having to
later add enable_parallel_update, should parallel update have
similar potential overhead in the parallel-safety checks (which to
me, looks like it could, and parallel delete may not ...)It's a shame there is no "set" type for GUC options.
e.g.
enable_parallel_dml='insert,update'
Maybe that's going too far.Isn't that just GUC_LIST_INPUT ?
I'm not sure why it'd be going to far ?The GUC-setting assign hook can parse the enable_parallel_dml_list value set
by the user, and set an internal int/bits enable_parallel_dml variable with some
define/enum values like:GUC_PARALLEL_DML_INSERT 0x01
GUC_PARALLEL_DML_DELETE 0x02
GUC_PARALLEL_DML_UPDATE 0x04I think this ideas works, but we still need to consider about the reloption.
After looking into the reloption, I think postgres do not have a list-like type for reloption.
And I think it's better that the guc and reloption is consistent.
I also think it is better to be consistent here.
Besides, a list type guc option that only support one valid value 'insert' seems a little weird to me(we only support parallel insert for now).
So, I tend to keep the current style of guc option.
+1. I feel at this stage it might not be prudent to predict the
overhead for parallel updates or deletes especially when there doesn't
appear to be an easy way to provide a futuristic guc/reloption and we
don't have any patch on the table which can prove or disprove that
theory. The only thing that we can see that even if parallel
updates/deletes have overhead, it might not be due to similar reasons.
Also, I guess the parallel-copy might need somewhat similar
parallel-safety checking w.r.t partitioned tables and I feel the
current proposed guc/reloption can be used for the same as it is quite
a similar operation.
--
With Regards,
Amit Kapila.
Greg Nancarrow <gregn4422@gmail.com> writes:
On Fri, Mar 12, 2021 at 5:00 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values
for those trigger functions?
... and also attached a patch to update the code for this issue.
Hm, what I was expecting to see is that RI_FKey_check_ins and
RI_FKey_check_upd get marked as proparallel = 'r' and the remainder
get marked as proparallel = 'u'. Remember that the default for
builtin functions is proparallel = 's', but surely that's wrong
for triggers that can propagate updates to other tables?
regards, tom lane
On Sat, Mar 13, 2021 at 10:08 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
Greg Nancarrow <gregn4422@gmail.com> writes:
On Fri, Mar 12, 2021 at 5:00 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
BTW, having special logic for FK triggers in
target_rel_trigger_max_parallel_hazard seems quite loony to me.
Why isn't that handled by setting appropriate proparallel values
for those trigger functions?... and also attached a patch to update the code for this issue.
Hm, what I was expecting to see is that RI_FKey_check_ins and
RI_FKey_check_upd get marked as proparallel = 'r' and the remainder
get marked as proparallel = 'u'.
oh, I think Greg's patch has just marked functions for which the
current code has parallel-safety checks and I also thought that would
be sufficient.
Remember that the default for
builtin functions is proparallel = 's', but surely that's wrong
for triggers that can propagate updates to other tables?
Okay, probably the others can be marked as unsafe. I think we have not
considered others except for FK-related triggers which we knew are
hazardous for enabling inserts in parallel-mode. The others seem to be
related to update/delete, so we have not done anything, but maybe it
is better to mark them as 'unsafe' now, and later when we support the
update/delete in parallel-mode, we can see if any of those can be
executed in parallel-mode. But OTOH, we can keep them as it is if they
don't impact the current operation we have supported in parallel-mode.
--
With Regards,
Amit Kapila.
On Fri, Mar 12, 2021 at 3:01 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
Attaching new version patch with this change.
Thanks, the patch looks good to me. I have made some minor cosmetic
changes in the attached. I am planning to push this by tomorrow unless
you or others have any more comments or suggestions for this patch.
--
With Regards,
Amit Kapila.
Attachments:
v28-0001-Add-a-new-GUC-and-a-reloption-to-enable-inserts-.patchapplication/octet-stream; name=v28-0001-Add-a-new-GUC-and-a-reloption-to-enable-inserts-.patchDownload
From cb0a9a2a34cb88a82bccb5a4b40140fc856ea741 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Mon, 15 Mar 2021 11:17:17 +0530
Subject: [PATCH v28] Add a new GUC and a reloption to enable inserts in
parallel-mode.
Commit 05c8482f7f added the implementation of parallel SELECT for
"INSERT INTO ... SELECT ..." which may incur non-negligible overhead in
the additional parallel-safety checks that it performs, even when, in the
end, those checks determine that parallelism can't be used. This is
normally only ever a problem in the case of when the target table has a
large number of partitions.
A new GUC option "enable_parallel_insert" is added, to allow insert in
parallel-mode. The default is on.
In addition to the GUC option, the user may want a mechanism to allow
inserts in parallel-mode with finer granularity at table level. The new
table option "parallel_insert_enabled" allows this. The default is true.
Author: "Hou, Zhijie"
Reviewed-by: Greg Nancarrow, Amit Langote, Takayuki Tsunakawa, Amit Kapila
Discussion: https://postgr.es/m/CAA4eK1K-cW7svLC2D7DHoGHxdAdg3P37BLgebqBOC2ZLc9a6QQ%40mail.gmail.com
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
doc/src/sgml/config.sgml | 23 +++++++++++
doc/src/sgml/ref/alter_table.sgml | 3 +-
doc/src/sgml/ref/create_table.sgml | 26 +++++++++++++
src/backend/access/common/reloptions.c | 25 +++++++++---
src/backend/optimizer/path/costsize.c | 2 +
src/backend/optimizer/util/clauses.c | 34 ++++++++++++++--
src/backend/utils/misc/guc.c | 10 +++++
src/backend/utils/misc/postgresql.conf.sample | 1 +
src/bin/psql/tab-complete.c | 1 +
src/include/optimizer/cost.h | 1 +
src/include/utils/rel.h | 25 ++++++++++++
src/test/regress/expected/insert_parallel.out | 56 ++++++++++++++++++++++++++-
src/test/regress/expected/sysviews.out | 3 +-
src/test/regress/sql/insert_parallel.sql | 44 ++++++++++++++++++++-
src/tools/pgindent/typedefs.list | 1 +
15 files changed, 240 insertions(+), 15 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a218d78..9ade5d5 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5072,6 +5072,29 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-enable-parallel-insert" xreflabel="enable_parallel_insert">
+ <term><varname>enable_parallel_insert</varname> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>enable_parallel_insert</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel plans for
+ <command>INSERT</command> commands. The default is <literal>on</literal>.
+ When enabled, the planner performs additional parallel-safety checks
+ on the target table's attributes and indexes, in order to determine
+ if it's safe to use a parallel plan for <command>INSERT</command>. In
+ cases such as when the target table has a large number of partitions,
+ and particularly also when that table uses something parallel-unsafe
+ that prevents parallelism, the overhead of these checks may become
+ prohibitively high. To address this potential overhead in these cases,
+ this option can be used to disable the use of parallel plans for
+ <command>INSERT</command>.
+ </para>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
<sect2 id="runtime-config-query-constants">
diff --git a/doc/src/sgml/ref/alter_table.sgml b/doc/src/sgml/ref/alter_table.sgml
index c25ef5a..f82dce4 100644
--- a/doc/src/sgml/ref/alter_table.sgml
+++ b/doc/src/sgml/ref/alter_table.sgml
@@ -722,7 +722,8 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
<para>
<literal>SHARE UPDATE EXCLUSIVE</literal> lock will be taken for
fillfactor, toast and autovacuum storage parameters, as well as the
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and
+ <varname>parallel_insert_enabled</varname>.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index 71703da..ff1b642 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1409,6 +1409,32 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
</listitem>
</varlistentry>
+ <varlistentry id="reloption-parallel-insert-enabled" xreflabel="parallel_insert_enabled">
+ <term><literal>parallel_insert_enabled</literal> (<type>boolean</type>)
+ <indexterm>
+ <primary><varname>parallel_insert_enabled</varname> storage parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Enables or disables the query planner's use of parallel insert for
+ this table. When enabled (and provided that
+ <xref linkend="guc-enable-parallel-insert"/> is also <literal>true</literal>),
+ the planner performs additional parallel-safety checks on the table's
+ attributes and indexes, in order to determine if it's safe to use a
+ parallel plan for <command>INSERT</command>. The default is
+ <literal>true</literal>. In cases such as when the table has a large
+ number of partitions, and particularly also when that table uses a
+ parallel-unsafe feature that prevents parallelism, the overhead of these
+ checks may become prohibitively high. To address this potential overhead
+ in these cases, this option can be used to disable the use of parallel
+ insert for this table. Note that if the target table of the parallel
+ insert is partitioned, the <literal>parallel_insert_enabled</literal>
+ option values of the partitions are ignored.
+ </para>
+ </listitem>
+ </varlistentry>
+
<varlistentry id="reloption-autovacuum-enabled" xreflabel="autovacuum_enabled">
<term><literal>autovacuum_enabled</literal>, <literal>toast.autovacuum_enabled</literal> (<type>boolean</type>)
<indexterm>
diff --git a/src/backend/access/common/reloptions.c b/src/backend/access/common/reloptions.c
index d897bbe..5a0ae99 100644
--- a/src/backend/access/common/reloptions.c
+++ b/src/backend/access/common/reloptions.c
@@ -168,6 +168,15 @@ static relopt_bool boolRelOpts[] =
},
true
},
+ {
+ {
+ "parallel_insert_enabled",
+ "Enables \"parallel insert\" feature for this table",
+ RELOPT_KIND_HEAP | RELOPT_KIND_PARTITIONED,
+ ShareUpdateExclusiveLock
+ },
+ true
+ },
/* list terminator */
{{NULL}}
};
@@ -1859,7 +1868,9 @@ default_reloptions(Datum reloptions, bool validate, relopt_kind kind)
{"vacuum_index_cleanup", RELOPT_TYPE_BOOL,
offsetof(StdRdOptions, vacuum_index_cleanup)},
{"vacuum_truncate", RELOPT_TYPE_BOOL,
- offsetof(StdRdOptions, vacuum_truncate)}
+ offsetof(StdRdOptions, vacuum_truncate)},
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(StdRdOptions, parallel_insert_enabled)}
};
return (bytea *) build_reloptions(reloptions, validate, kind,
@@ -1961,13 +1972,15 @@ build_local_reloptions(local_relopts *relopts, Datum options, bool validate)
bytea *
partitioned_table_reloptions(Datum reloptions, bool validate)
{
- /*
- * There are no options for partitioned tables yet, but this is able to do
- * some validation.
- */
+ static const relopt_parse_elt tab[] = {
+ {"parallel_insert_enabled", RELOPT_TYPE_BOOL,
+ offsetof(PartitionedTableRdOptions, parallel_insert_enabled)}
+ };
+
return (bytea *) build_reloptions(reloptions, validate,
RELOPT_KIND_PARTITIONED,
- 0, NULL, 0);
+ sizeof(PartitionedTableRdOptions),
+ tab, lengthof(tab));
}
/*
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index a25b674..c81e2cf 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -129,6 +129,8 @@ Cost disable_cost = 1.0e10;
int max_parallel_workers_per_gather = 2;
+bool enable_parallel_insert = true;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
diff --git a/src/backend/optimizer/util/clauses.c b/src/backend/optimizer/util/clauses.c
index ade66a5..c6be4f8 100644
--- a/src/backend/optimizer/util/clauses.c
+++ b/src/backend/optimizer/util/clauses.c
@@ -1265,8 +1265,10 @@ target_rel_chk_constr_max_parallel_hazard(Relation rel,
*
* It's not possible in the following cases:
*
- * 1) INSERT...ON CONFLICT...DO UPDATE
- * 2) INSERT without SELECT
+ * 1) enable_parallel_insert is off
+ * 2) INSERT...ON CONFLICT...DO UPDATE
+ * 3) INSERT without SELECT
+ * 4) the reloption parallel_insert_enabled is set to off
*
* (Note: we don't do in-depth parallel-safety checks here, we do only the
* cheaper tests that can quickly exclude obvious cases for which
@@ -1277,12 +1279,17 @@ bool
is_parallel_allowed_for_modify(Query *parse)
{
bool hasSubQuery;
+ bool parallel_enabled;
RangeTblEntry *rte;
ListCell *lc;
+ Relation rel;
if (!IsModifySupportedInParallelMode(parse->commandType))
return false;
+ if (!enable_parallel_insert)
+ return false;
+
/*
* UPDATE is not currently supported in parallel-mode, so prohibit
* INSERT...ON CONFLICT...DO UPDATE...
@@ -1313,7 +1320,28 @@ is_parallel_allowed_for_modify(Query *parse)
}
}
- return hasSubQuery;
+ if (!hasSubQuery)
+ return false;
+
+ /*
+ * Check if parallel_insert_enabled is enabled for the target table, if
+ * not, skip the safety checks.
+ *
+ * (Note: if the target table is partitioned, the parallel_insert_enabled
+ * option setting of the partitions are ignored).
+ */
+ rte = rt_fetch(parse->resultRelation, parse->rtable);
+
+ /*
+ * The target table is already locked by the caller (this is done in the
+ * parse/analyze phase), and remains locked until end-of-transaction.
+ */
+ rel = table_open(rte->relid, NoLock);
+
+ parallel_enabled = RelationGetParallelInsert(rel, true);
+ table_close(rel, NoLock);
+
+ return parallel_enabled;
}
/*****************************************************************************
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 855076b..b263e34 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -1123,6 +1123,16 @@ static struct config_bool ConfigureNamesBool[] =
NULL, NULL, NULL
},
{
+ {"enable_parallel_insert", PGC_USERSET, QUERY_TUNING_METHOD,
+ gettext_noop("Enables the planner's use of parallel plans for INSERT commands."),
+ NULL,
+ GUC_EXPLAIN
+ },
+ &enable_parallel_insert,
+ true,
+ NULL, NULL, NULL
+ },
+ {
/* Not for general use --- used by SET SESSION AUTHORIZATION */
{"is_superuser", PGC_INTERNAL, UNGROUPED,
gettext_noop("Shows whether the current user is a superuser."),
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index f46c2dd..6647f8f 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -371,6 +371,7 @@
#enable_partitionwise_aggregate = off
#enable_parallel_hash = on
#enable_partition_pruning = on
+#enable_parallel_insert = on
# - Planner Cost Constants -
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index ecdb8d7..d3fb734 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1118,6 +1118,7 @@ static const char *const table_storage_parameters[] = {
"autovacuum_vacuum_threshold",
"fillfactor",
"log_autovacuum_min_duration",
+ "parallel_insert_enabled",
"parallel_workers",
"toast.autovacuum_enabled",
"toast.autovacuum_freeze_max_age",
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 1be93be..22e6db9 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -47,6 +47,7 @@ typedef enum
/* parameter variables and flags (see also optimizer.h) */
extern PGDLLIMPORT Cost disable_cost;
extern PGDLLIMPORT int max_parallel_workers_per_gather;
+extern PGDLLIMPORT bool enable_parallel_insert;
extern PGDLLIMPORT bool enable_seqscan;
extern PGDLLIMPORT bool enable_indexscan;
extern PGDLLIMPORT bool enable_indexonlyscan;
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 10b6398..5375a37 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -306,6 +306,8 @@ typedef struct StdRdOptions
int parallel_workers; /* max number of parallel workers */
bool vacuum_index_cleanup; /* enables index vacuuming and cleanup */
bool vacuum_truncate; /* enables vacuum to truncate a relation */
+ bool parallel_insert_enabled; /* enables planner's use of
+ * parallel insert */
} StdRdOptions;
#define HEAP_MIN_FILLFACTOR 10
@@ -424,6 +426,29 @@ typedef struct ViewOptions
VIEW_OPTION_CHECK_OPTION_CASCADED)
/*
+ * PartitionedTableRdOptions
+ * Contents of rd_options for partitioned tables
+ */
+typedef struct PartitionedTableRdOptions
+{
+ int32 vl_len_; /* varlena header (do not touch directly!) */
+ bool parallel_insert_enabled; /* enables planner's use of
+ * parallel insert */
+} PartitionedTableRdOptions;
+
+/*
+ * RelationGetParallelInsert
+ * Returns the relation's parallel_insert_enabled reloption setting.
+ * Note multiple eval of argument!
+ */
+#define RelationGetParallelInsert(relation, defaultpd) \
+ ((relation)->rd_options ? \
+ (relation->rd_rel->relkind == RELKIND_PARTITIONED_TABLE ? \
+ ((PartitionedTableRdOptions *) (relation)->rd_options)->parallel_insert_enabled : \
+ ((StdRdOptions *) (relation)->rd_options)->parallel_insert_enabled) : \
+ (defaultpd))
+
+/*
* RelationIsValid
* True iff relation descriptor is valid.
*/
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index d5fae79..cb89453 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -63,12 +63,45 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+ QUERY PLAN
+--------------------------
+ Insert on para_insert_p1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
+--
-- Test INSERT with underlying query.
-- (should create plan with parallel SELECT, Gather parent node)
--
@@ -364,9 +397,28 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+ QUERY PLAN
+-------------------------
+ Insert on parttable1
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out
index 6d048e3..a62bf5d 100644
--- a/src/test/regress/expected/sysviews.out
+++ b/src/test/regress/expected/sysviews.out
@@ -107,13 +107,14 @@ select name, setting from pg_settings where name like 'enable%';
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
+ enable_parallel_insert | on
enable_partition_pruning | on
enable_partitionwise_aggregate | off
enable_partitionwise_join | off
enable_seqscan | on
enable_sort | on
enable_tidscan | on
-(18 rows)
+(19 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/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 70ad31a..5092990 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -79,13 +79,38 @@ set max_parallel_workers_per_gather=4;
create table para_insert_p1 (
unique1 int4 PRIMARY KEY,
stringu1 name
-);
+) with (parallel_insert_enabled = off);
create table para_insert_f1 (
unique1 int4 REFERENCES para_insert_p1(unique1),
stringu1 name
);
+--
+-- Disable guc option enable_parallel_insert
+--
+set enable_parallel_insert = off;
+
+-- Test INSERT with underlying query when enable_parallel_insert=off and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable guc option enable_parallel_insert
+--
+set enable_parallel_insert = on;
+
+--
+-- Test INSERT with underlying query when enable_parallel_insert=on and reloption.parallel_insert_enabled=off.
+-- (should create plan with serial INSERT + SELECT)
+--
+explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
@@ -210,10 +235,25 @@ explain (costs off) insert into testdef(a,d) select a,a*8 from test_data;
--
-- Test INSERT into partition with underlying query.
--
-create table parttable1 (a int, b name) partition by range (a);
+create table parttable1 (a int, b name) partition by range (a) with (parallel_insert_enabled=off);
create table parttable1_1 partition of parttable1 for values from (0) to (5000);
create table parttable1_2 partition of parttable1 for values from (5000) to (10000);
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=off
+-- (should not create a parallel plan)
+--
+explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
+
+--
+-- Enable reloption parallel_insert_enabled
+--
+alter table parttable1 set (parallel_insert_enabled = on);
+
+--
+-- Test INSERT into partition when reloption.parallel_insert_enabled=on
+-- (should create a parallel plan)
+--
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 61cf4ea..c8abd73 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -1795,6 +1795,7 @@ PartitionSpec
PartitionTupleRouting
PartitionedRelPruneInfo
PartitionedRelPruningData
+PartitionedTableRdOptions
PartitionwiseAggregateType
PasswordType
Path
--
1.8.3.1
On Mon, Mar 15, 2021 at 11:25:26AM +0530, Amit Kapila wrote:
On Fri, Mar 12, 2021 at 3:01 PM houzj.fnst@fujitsu.com wrote:
Attaching new version patch with this change.
Thanks, the patch looks good to me. I have made some minor cosmetic
changes in the attached. I am planning to push this by tomorrow unless
you or others have any more comments or suggestions for this patch.
I still wonder if it should be possible for the GUC to be off, and then
selectively enable parallel inserts on specific tables. In that case, the GUC
should be called something other than enable_*, or maybe it should be an enum
with values like "off" and "allowed"/enabled/defer/???
--
Justin
On Mon, Mar 15, 2021 at 11:34 AM Justin Pryzby <pryzby@telsasoft.com> wrote:
On Mon, Mar 15, 2021 at 11:25:26AM +0530, Amit Kapila wrote:
On Fri, Mar 12, 2021 at 3:01 PM houzj.fnst@fujitsu.com wrote:
Attaching new version patch with this change.
Thanks, the patch looks good to me. I have made some minor cosmetic
changes in the attached. I am planning to push this by tomorrow unless
you or others have any more comments or suggestions for this patch.I still wonder if it should be possible for the GUC to be off, and then
selectively enable parallel inserts on specific tables.
I think that could be inconvenient for users because in most tables
such a restriction won't need to be applied.
--
With Regards,
Amit Kapila.
Attaching new version patch with this change.
Thanks, the patch looks good to me. I have made some minor cosmetic
changes in the attached. I am planning to push this by tomorrow unless you or
others have any more comments or suggestions for this patch.
Thanks for the review.
I have no more comments.
BTW, I have done the final test on the patch and all of the tests passed.
Best regards,
houzj
On Fri, Mar 12, 2021 at 04:05:09PM +0900, Amit Langote wrote:
On Fri, Mar 12, 2021 at 6:10 AM Justin Pryzby <pryzby@telsasoft.com> wrote:
Note also this CF entry
https://commitfest.postgresql.org/32/2987/
| Allow setting parallel_workers on partitioned tables
I'm rebasing that other patch on top of master (with this patch), and I noticed
that it adds this bit, and this patch should have done something similar:
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1340,4 +1340,5 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
will use the table's parameter value.
- Specifying these parameters for partitioned tables is not supported,
- but you may specify them for individual leaf partitions.
+ These parameters, with the exception of <literal>parallel_workers</literal>,
+ are not supported on partitioned tables, but you may specify them for
+ individual leaf partitions.
</para>
--
Justin
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index ff1b642722..d5d356f2de 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1338,8 +1338,10 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
If a table parameter value is set and the
equivalent <literal>toast.</literal> parameter is not, the TOAST table
will use the table's parameter value.
- Specifying these parameters for partitioned tables is not supported,
- but you may specify them for individual leaf partitions.
+ These parameters, with the exception of
+ <literal>parallel_insert_enabled</literal>,
+ are not supported on partitioned tables, but may be specified for
+ individual leaf partitions.
</para>
<variablelist>
Attachments:
0001-Fix-for-parallel_insert_enabled.patchtext/x-diff; charset=us-asciiDownload
From 55c7326e56f9b49710a564e91c46212a17f12b24 Mon Sep 17 00:00:00 2001
From: Justin Pryzby <pryzbyj@telsasoft.com>
Date: Wed, 17 Mar 2021 21:47:27 -0500
Subject: [PATCH] Fix for parallel_insert_enabled
---
doc/src/sgml/ref/create_table.sgml | 6 ++++--
1 file changed, 4 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml
index ff1b642722..d5d356f2de 100644
--- a/doc/src/sgml/ref/create_table.sgml
+++ b/doc/src/sgml/ref/create_table.sgml
@@ -1338,8 +1338,10 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM
If a table parameter value is set and the
equivalent <literal>toast.</literal> parameter is not, the TOAST table
will use the table's parameter value.
- Specifying these parameters for partitioned tables is not supported,
- but you may specify them for individual leaf partitions.
+ These parameters, with the exception of
+ <literal>parallel_insert_enabled</literal>,
+ are not supported on partitioned tables, but may be specified for
+ individual leaf partitions.
</para>
<variablelist>
--
2.17.0
On Thu, Mar 18, 2021 at 8:30 AM Justin Pryzby <pryzby@telsasoft.com> wrote:
diff --git a/doc/src/sgml/ref/create_table.sgml b/doc/src/sgml/ref/create_table.sgml index ff1b642722..d5d356f2de 100644 --- a/doc/src/sgml/ref/create_table.sgml +++ b/doc/src/sgml/ref/create_table.sgml @@ -1338,8 +1338,10 @@ WITH ( MODULUS <replaceable class="parameter">numeric_literal</replaceable>, REM If a table parameter value is set and the equivalent <literal>toast.</literal> parameter is not, the TOAST table will use the table's parameter value. - Specifying these parameters for partitioned tables is not supported, - but you may specify them for individual leaf partitions. + These parameters, with the exception of + <literal>parallel_insert_enabled</literal>, + are not supported on partitioned tables, but may be specified for + individual leaf partitions. </para>
Your patch looks good to me. While checking this, I notice a typo in
the previous patch:
- planner parameter <varname>parallel_workers</varname>.
+ planner parameter <varname>parallel_workers</varname> and
+ <varname>parallel_insert_enabled</varname>.
Here, it should be /planner parameter/planner parameters.
--
With Regards,
Amit Kapila.
If a table parameter value is set and the equivalent <literal>toast.</literal> parameter is not, the TOAST table will use the table's parameter value. - Specifying these parameters for partitioned tables is not supported, - but you may specify them for individual leaf partitions. + These parameters, with the exception of + <literal>parallel_insert_enabled</literal>, + are not supported on partitioned tables, but may be specified for + individual leaf partitions. </para>Your patch looks good to me. While checking this, I notice a typo in the previous patch: - planner parameter <varname>parallel_workers</varname>. + planner parameter <varname>parallel_workers</varname> and + <varname>parallel_insert_enabled</varname>.Here, it should be /planner parameter/planner parameters.
Thanks amit and justin for pointing this out !
The changes looks good to me.
Best regards,
houzj
On Thu, Mar 18, 2021 at 9:04 AM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
If a table parameter value is set and the equivalent <literal>toast.</literal> parameter is not, the TOAST table will use the table's parameter value. - Specifying these parameters for partitioned tables is not supported, - but you may specify them for individual leaf partitions. + These parameters, with the exception of + <literal>parallel_insert_enabled</literal>, + are not supported on partitioned tables, but may be specified for + individual leaf partitions. </para>Your patch looks good to me. While checking this, I notice a typo in the previous patch: - planner parameter <varname>parallel_workers</varname>. + planner parameter <varname>parallel_workers</varname> and + <varname>parallel_insert_enabled</varname>.Here, it should be /planner parameter/planner parameters.
Thanks amit and justin for pointing this out !
The changes looks good to me.
Pushed!
--
With Regards,
Amit Kapila.
Since the "Parallel SELECT for INSERT" patch and related GUC/reloption
patch have been committed, I have now rebased and attached the rest of the
original patchset, which includes:
- Additional tests for "Parallel SELECT for INSERT"
- Parallel INSERT functionality
- Test and documentation updates for Parallel INSERT
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v29-0001-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v29-0001-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From 4ad712c2cbe1c9038bcfa32794b9dbd3f7c4e389 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH 1/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 164668e319..2fef07bb6c 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -180,6 +202,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -203,6 +252,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -227,6 +356,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -323,6 +654,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -343,6 +719,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -381,6 +791,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -439,6 +878,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -457,16 +954,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -479,18 +984,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -501,8 +1015,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -551,15 +1100,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -568,6 +1156,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -575,6 +1198,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -583,4 +1208,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 171d8e5b84..cd8f9ef00e 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -141,6 +167,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -152,6 +186,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -161,6 +218,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -197,6 +318,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -211,6 +357,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -225,6 +384,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -257,6 +424,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -271,37 +478,78 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -335,6 +583,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -342,15 +601,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -361,6 +653,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -370,4 +664,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v29-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v29-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 1538f6833eeb75ce755046c12ea0ca99c7ad0a92 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 4 Mar 2021 22:44:54 +1100
Subject: [PATCH 2/3] Enable parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 8 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 +++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 ++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 269 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 23 ++
src/backend/optimizer/util/pathnode.c | 56 ++---
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
18 files changed, 512 insertions(+), 162 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 7cb87f4a3b..76121a8c95 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2294,10 +2297,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 6395a9b240..9997e07e06 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -516,6 +516,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -764,12 +778,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1020,12 +1038,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a TransactionId.
* Failure to do this now would result in heap_insert() subsequently
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..9510ebc83f 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -776,7 +776,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1514,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c81e2cf244..7618737ea2 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 906cab7053..2a3ad42f90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -345,7 +345,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 424d25cbd5..03b57771fc 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -323,10 +326,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1807,7 +1811,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1855,6 +1972,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2295,96 +2413,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2403,6 +2458,18 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ /*
+ * Generate gather paths according to the added partial paths for the
+ * parallel table-modification command.
+ * Note that true is passed for the "override_rows" parameter, so that
+ * the rows from the cheapest partial path (ModifyTablePath) are used,
+ * not the rel's (possibly estimated) rows.
+ */
+ generate_useful_gather_paths(root, final_rel, true);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7572,7 +7639,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 42f088ad71..4d7773538e 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -1066,6 +1066,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 69b83071cf..e6af7d3067 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3553,6 +3553,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3563,10 +3564,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3583,47 +3584,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3639,6 +3615,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e31ad6204e..27272ec2c8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 22e6db96b6..a6a53599cd 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 54f4b782fc..cb36413e5b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -267,7 +267,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 5375a37dd1..9b8d459e9c 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -600,15 +601,15 @@ typedef struct PartitionedTableRdOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v29-0003-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v29-0003-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From 340421661d42e5f17fab9116bae916d8063cbfdf Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 18 Mar 2021 15:18:35 +1100
Subject: [PATCH 3/3] INSERT and/or SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 81 +++++++++-
src/test/regress/expected/insert_parallel.out | 146 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 28 ++--
3 files changed, 164 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 330c06b608..ffdbd91701 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,12 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the following commands which create a new table and populate
- it can use a parallel plan for the underlying <literal>SELECT</literal>
- part of the query:
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
<itemizedlist>
<listitem>
@@ -442,6 +446,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 2fef07bb6c..2c0c42b11c 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -123,14 +123,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -151,7 +151,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -183,15 +183,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -261,9 +261,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -300,9 +300,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -340,9 +340,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -389,9 +389,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -424,9 +424,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -460,9 +460,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -498,9 +498,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -538,9 +538,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -584,21 +584,21 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -627,7 +627,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -663,9 +663,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -725,14 +725,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -764,7 +764,7 @@ explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
(2 rows)
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -793,14 +793,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ alter table parttable1 set (parallel_insert_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -955,7 +955,8 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan; triggers should fire)
+-- (should create a parallel INSERT+SELECT plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
create or replace function insert_before_trigger_safe() returns trigger as $$
@@ -975,11 +976,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -988,7 +989,8 @@ NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
--- (should not create a parallel plan; triggers should fire)
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
create or replace function insert_before_trigger_unsafe() returns trigger as $$
@@ -1019,7 +1021,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1076,13 +1078,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1109,9 +1111,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index cd8f9ef00e..5cb96c8c60 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -138,7 +138,7 @@ alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -149,7 +149,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -161,7 +161,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -297,14 +297,14 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -312,7 +312,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -364,7 +364,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -377,7 +377,7 @@ truncate testdef;
explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -385,7 +385,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -479,7 +479,8 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan; triggers should fire)
+-- (should create a parallel INSERT+SELECT plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
create or replace function insert_before_trigger_safe() returns trigger as $$
@@ -503,7 +504,8 @@ insert into names_with_safe_trigger select * from names;
--
-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
--- (should not create a parallel plan; triggers should fire)
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
create or replace function insert_before_trigger_unsafe() returns trigger as $$
@@ -527,7 +529,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -573,7 +575,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
Since the "Parallel SELECT for INSERT" patch and related GUC/reloption patch have been committed, I have now rebased and attached the rest of the original patchset,
which includes:
- Additional tests for "Parallel SELECT for INSERT"
- Parallel INSERT functionality
- Test and documentation updates for Parallel INSERT
Hi,
I noticed that some comments may need updated since we introduced parallel insert in this patch.
1) src/backend/executor/execMain.c
* Don't allow writes in parallel mode. Supporting UPDATE and DELETE
* would require (a) storing the combocid hash in shared memory, rather
* than synchronizing it just once at the start of parallelism, and (b) an
* alternative to heap_update()'s reliance on xmax for mutual exclusion.
* INSERT may have no such troubles, but we forbid it to simplify the
* checks.
As we will allow INSERT in parallel mode, we'd better change the comment here.
2) src/backend/storage/lmgr/README
dangers are modest. The leader and worker share the same transaction,
snapshot, and combo CID hash, and neither can perform any DDL or, indeed,
write any data at all. Thus, for either to read a table locked exclusively by
The same as 1), parallel insert is the exception.
3) src/backend/storage/lmgr/README
mutual exclusion method for such cases. Currently, the parallel mode is
strictly read-only, but now we have the infrastructure to allow parallel
inserts and parallel copy.
May be we can say:
+mutual exclusion method for such cases. Currently, we only allowed parallel
+inserts, but we already have the infrastructure to allow parallel copy.
Best regards,
houzj
On Mon, Mar 22, 2021 at 2:30 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
I noticed that some comments may need updated since we introduced parallel insert in this patch.
1) src/backend/executor/execMain.c
* Don't allow writes in parallel mode. Supporting UPDATE and DELETE
* would require (a) storing the combocid hash in shared memory, rather
* than synchronizing it just once at the start of parallelism, and (b) an
* alternative to heap_update()'s reliance on xmax for mutual exclusion.
* INSERT may have no such troubles, but we forbid it to simplify the
* checks.As we will allow INSERT in parallel mode, we'd better change the comment here.
Thanks, it does need to be updated for parallel INSERT.
I was thinking of the following change:
- * Don't allow writes in parallel mode. Supporting UPDATE and DELETE
- * would require (a) storing the combocid hash in shared memory, rather
- * than synchronizing it just once at the start of parallelism, and (b) an
- * alternative to heap_update()'s reliance on xmax for mutual exclusion.
- * INSERT may have no such troubles, but we forbid it to simplify the
- * checks.
+ * Except for INSERT, don't allow writes in parallel mode. Supporting
+ * UPDATE and DELETE would require (a) storing the combocid hash in shared
+ * memory, rather than synchronizing it just once at the start of
+ * parallelism, and (b) an alternative to heap_update()'s reliance on xmax
+ * for mutual exclusion.
2) src/backend/storage/lmgr/README
dangers are modest. The leader and worker share the same transaction,
snapshot, and combo CID hash, and neither can perform any DDL or, indeed,
write any data at all. Thus, for either to read a table locked exclusively byThe same as 1), parallel insert is the exception.
I agree, it needs to be updated too, to account for parallel INSERT
now being supported.
-write any data at all. ...
+write any data at all (with the exception of parallel insert). ...
3) src/backend/storage/lmgr/README
mutual exclusion method for such cases. Currently, the parallel mode is
strictly read-only, but now we have the infrastructure to allow parallel
inserts and parallel copy.May be we can say: +mutual exclusion method for such cases. Currently, we only allowed parallel +inserts, but we already have the infrastructure to allow parallel copy.
Yes, agree, something like:
-mutual exclusion method for such cases. Currently, the parallel mode is
-strictly read-only, but now we have the infrastructure to allow parallel
-inserts and parallel copy.
+mutual exclusion method for such cases. Currently, only parallel insert is
+allowed, but we have the infrastructure to allow parallel copy.
Let me know if these changes seem OK to you.
Regards,
Greg Nancarrow
Fujitsu Australia
I noticed that some comments may need updated since we introduced
parallel insert in this patch.
1) src/backend/executor/execMain.c
* Don't allow writes in parallel mode. Supporting UPDATE andDELETE
* would require (a) storing the combocid hash in shared memory,
rather
* than synchronizing it just once at the start of parallelism, and (b) an
* alternative to heap_update()'s reliance on xmax for mutualexclusion.
* INSERT may have no such troubles, but we forbid it to simplify the
* checks.As we will allow INSERT in parallel mode, we'd better change the comment
here.
Thanks, it does need to be updated for parallel INSERT.
I was thinking of the following change:- * Don't allow writes in parallel mode. Supporting UPDATE and DELETE - * would require (a) storing the combocid hash in shared memory, rather - * than synchronizing it just once at the start of parallelism, and (b) an - * alternative to heap_update()'s reliance on xmax for mutual exclusion. - * INSERT may have no such troubles, but we forbid it to simplify the - * checks. + * Except for INSERT, don't allow writes in parallel mode. Supporting + * UPDATE and DELETE would require (a) storing the combocid hash in shared + * memory, rather than synchronizing it just once at the start of + * parallelism, and (b) an alternative to heap_update()'s reliance on xmax + * for mutual exclusion.2) src/backend/storage/lmgr/README
dangers are modest. The leader and worker share the same transaction,
snapshot, and combo CID hash, and neither can perform any DDL or,
indeed, write any data at all. Thus, for either to read a table
locked exclusively byThe same as 1), parallel insert is the exception.
I agree, it needs to be updated too, to account for parallel INSERT now being
supported.-write any data at all. ... +write any data at all (with the exception of parallel insert). ...3) src/backend/storage/lmgr/README
mutual exclusion method for such cases. Currently, the parallel mode
is strictly read-only, but now we have the infrastructure to allow
parallel inserts and parallel copy.May be we can say: +mutual exclusion method for such cases. Currently, we only allowed +parallel inserts, but we already have the infrastructure to allow parallel copy.Yes, agree, something like:
-mutual exclusion method for such cases. Currently, the parallel mode is -strictly read-only, but now we have the infrastructure to allow parallel -inserts and parallel copy. +mutual exclusion method for such cases. Currently, only parallel +insert is allowed, but we have the infrastructure to allow parallel copy.Let me know if these changes seem OK to you.
Yes, these changes look good to me.
Best regards,
houzj
On Mon, Mar 22, 2021 at 6:28 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:
Let me know if these changes seem OK to you.
Yes, these changes look good to me.
Posting an updated set of patches with these changes...
Regards,
Greg Nancarrow
Fujitsu Australia
Attachments:
v30-0001-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchapplication/octet-stream; name=v30-0001-Parallel-SELECT-for-INSERT-INTO-.-SELECT-advanced-tests.patchDownload
From 4342bf435612b089cd403de39633b56d46462e95 Mon Sep 17 00:00:00 2001
From: Amit Kapila <akapila@postgresql.org>
Date: Sat, 6 Mar 2021 15:23:57 +0530
Subject: [PATCH 1/3] Parallel SELECT for "INSERT INTO ... SELECT ..."
-advanced tests.
---
src/test/regress/expected/insert_parallel.out | 633 +++++++++++++++++-
src/test/regress/sql/insert_parallel.sql | 302 ++++++++-
2 files changed, 931 insertions(+), 4 deletions(-)
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 164668e319..2fef07bb6c 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -11,14 +11,36 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
return f || l;
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
insert into names values
@@ -180,6 +202,33 @@ insert into test_data1 select * from test_data where a = 10 returning a as data;
10
(1 row)
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ QUERY PLAN
+--------------------------------------------------
+ Insert on test_data1
+ -> Gather Merge
+ Workers Planned: 3
+ -> Sort
+ Sort Key: test_data.a
+ -> Parallel Seq Scan on test_data
+ Filter: (a <= 5)
+(7 rows)
+
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+ data
+------
+ 1
+ 2
+ 3
+ 4
+ 5
+(5 rows)
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -203,6 +252,86 @@ select count(*), sum(unique1) from para_insert_f1;
10000 | 49995000
(1 row)
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+ QUERY PLAN
+-----------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+ Filter: (unique1 <= 2500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+---------
+ 2501 | 3126250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 2490 | URAAAA
+ 2491 | VRAAAA
+ 2492 | WRAAAA
+ 2493 | XRAAAA
+ 2494 | YRAAAA
+ 2495 | ZRAAAA
+ 2496 | ASAAAA
+ 2497 | BSAAAA
+ 2498 | CSAAAA
+ 2499 | DSAAAA
+ 2500 | ESAAAA
+(11 rows)
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -227,6 +356,208 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
-> Seq Scan on test_data
(4 rows)
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on tenk1_avg_data
+ -> Subquery Scan on "*SELECT*"
+ -> Finalize Aggregate
+ -> Gather
+ Workers Planned: 4
+ -> Partial Aggregate
+ -> Parallel Seq Scan on tenk1
+(7 rows)
+
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+ count | avg_unique1 | avg_stringu1_len
+-------+-------------+------------------
+ 10000 | 5000 | 6
+(1 row)
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+ QUERY PLAN
+------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Bitmap Heap Scan on tenk1
+ Recheck Cond: (unique1 >= 7500)
+ -> Bitmap Index Scan on tenk1_unique1
+ Index Cond: (unique1 >= 7500)
+(7 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+reset enable_seqscan;
+reset enable_indexscan;
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+ QUERY PLAN
+--------------------------------------------------------
+ Insert on a_star_data
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Append
+ -> Parallel Seq Scan on d_star a_star_4
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on f_star a_star_6
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on e_star a_star_5
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on b_star a_star_2
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on c_star a_star_3
+ Filter: (aa > 10)
+ -> Parallel Seq Scan on a_star a_star_1
+ Filter: (aa > 10)
+(16 rows)
+
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+ count | sum
+-------+-----
+ 16 | 300
+(1 row)
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+--------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1 | stringu1
+---------+----------
+ 9990 | GUAAAA
+ 9991 | HUAAAA
+ 9992 | IUAAAA
+ 9993 | JUAAAA
+ 9994 | KUAAAA
+ 9995 | LUAAAA
+ 9996 | MUAAAA
+ 9997 | NUAAAA
+ 9998 | OUAAAA
+ 9999 | PUAAAA
+(10 rows)
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+NOTICE: truncate cascades to table "para_insert_f1"
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+ QUERY PLAN
+-------------------------------------------------------------------
+ Insert on para_insert_p1
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Index Only Scan using tenk1_unique1 on tenk1
+ Index Cond: (unique1 >= 500)
+(5 rows)
+
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+ count | sum
+-------+----------
+ 9500 | 49870250
+(1 row)
+
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+ unique1
+---------
+ 9990
+ 9991
+ 9992
+ 9993
+ 9994
+ 9995
+ 9996
+ 9997
+ 9998
+ 9999
+(10 rows)
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names3
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
+ index | first_name | last_name
+-------+------------+-------------
+ 7 | alan | turing
+ 1 | albert | einstein
+ 3 | erwin | schrodinger
+ 6 | isaac | newton
+ 4 | leonhard | euler
+ 2 | niels | bohr
+ 8 | richard | feynman
+ 5 | stephen | hawking
+(8 rows)
+
--
-- Test INSERT with parallel-unsafe index expression
-- (should not create a parallel plan)
@@ -323,6 +654,51 @@ insert into names7 select * from names order by last_name returning last_name ||
turing, alan
(8 rows)
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names8
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names8 select * from names;
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+ QUERY PLAN
+-------------------------
+ Insert on names9
+ -> Seq Scan on names
+(2 rows)
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+ QUERY PLAN
+----------------------------------------
+ Insert on names10
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
-- (Insert into a temp table is parallel-restricted;
@@ -343,6 +719,40 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 8
+ 2 | 4 | 8 | 16
+ 3 | 6 | 12 | 24
+ 4 | 8 | 16 | 32
+ 5 | 10 | 20 | 40
+ 6 | 12 | 24 | 48
+ 7 | 14 | 28 | 56
+ 8 | 16 | 32 | 64
+ 9 | 18 | 36 | 72
+ 10 | 20 | 40 | 80
+(10 rows)
+
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
--
@@ -381,6 +791,35 @@ select * from testdef order by a;
10 | 20 | 10 | 80
(10 rows)
+truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+ QUERY PLAN
+--------------------------------------------
+ Insert on testdef
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on test_data
+(4 rows)
+
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+ a | b | c | d
+----+----+----+----
+ 1 | 2 | 4 | 20
+ 2 | 4 | 8 | 20
+ 3 | 6 | 12 | 20
+ 4 | 8 | 16 | 20
+ 5 | 10 | 20 | 20
+ 6 | 12 | 24 | 20
+ 7 | 14 | 28 | 20
+ 8 | 16 | 32 | 20
+ 9 | 18 | 36 | 20
+ 10 | 20 | 40 | 20
+(10 rows)
+
truncate testdef;
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
@@ -439,6 +878,64 @@ select count(*) from parttable1_2;
5000
(1 row)
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on partkey_unsafe_key_supp_fn_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+ QUERY PLAN
+-------------------------------------
+ Insert on partkey_unsafe_key_expr_t
+ -> Seq Scan on tenk1
+(2 rows)
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on table_check_a
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+ count | sum
+-------+----------
+ 10000 | 49995000
+(1 row)
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -457,16 +954,24 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
(2 rows)
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
@@ -479,18 +984,27 @@ explain (costs off) insert into names_with_safe_trigger select * from names;
(4 rows)
insert into names_with_safe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
@@ -501,8 +1015,43 @@ explain (costs off) insert into names_with_unsafe_trigger select * from names;
(2 rows)
insert into names_with_unsafe_trigger select * from names;
+NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+ QUERY PLAN
+-----------------------------------------
+ Insert on names_with_restricted_trigger
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on names
+(4 rows)
+
+insert into names_with_restricted_trigger select * from names;
+NOTICE: hello from insert_before_trigger_restricted
+NOTICE: hello from insert_after_trigger_restricted
+--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
--
@@ -551,15 +1100,54 @@ explain (costs off) execute q;
Filter: ((a % 2) = 0)
(3 rows)
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+ QUERY PLAN
+----------------------------------------------------------
+ Insert on insert_toast_table
+ -> Gather
+ Workers Planned: 3
+ -> Parallel Seq Scan on insert_toast_table_data
+(4 rows)
+
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+ row_count | total_data_length
+-----------+-------------------
+ 20 | 327680
+(1 row)
+
+truncate insert_toast_table;
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
QUERY PLAN
@@ -568,6 +1156,41 @@ explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
-> Seq Scan on tenk1
(2 rows)
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_r
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+ QUERY PLAN
+----------------------------------------
+ Insert on dom_table_s
+ -> Gather
+ Workers Planned: 4
+ -> Parallel Seq Scan on tenk1
+(4 rows)
+
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+ count | sum_x | sum_y
+-------+----------+----------
+ 10000 | 49995000 | 49995000
+(1 row)
+
rollback;
--
-- Clean up anything not created in the transaction
@@ -575,6 +1198,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -583,4 +1208,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index 171d8e5b84..cd8f9ef00e 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -15,15 +15,41 @@ create or replace function fullname_parallel_unsafe(f text, l text) returns text
end;
$$ language plpgsql immutable parallel unsafe;
+create or replace function fullname_parallel_safe(f text, l text) returns text as $$
+ begin
+ return f || l;
+ end;
+$$ language plpgsql immutable parallel safe;
+
create or replace function fullname_parallel_restricted(f text, l text) returns text as $$
begin
return f || l;
end;
$$ language plpgsql immutable parallel restricted;
+create or replace function lastname_startswithe_u(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel unsafe;
+
+create or replace function lastname_startswithe_s(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel safe;
+
+create or replace function lastname_startswithe_r(last_name text) returns boolean as $$
+ begin
+ return substring(last_name from 1 for 1) = 'e';
+ end;
+$$ language plpgsql immutable parallel restricted;
+
create table names(index int, first_name text, last_name text);
create table names2(index int, first_name text, last_name text);
create index names2_fullname_idx on names2 (fullname_parallel_unsafe(first_name, last_name));
+create table names3(index int, first_name text, last_name text);
+create index names3_fullname_idx on names3 (fullname_parallel_safe(first_name, last_name));
create table names4(index int, first_name text, last_name text);
create index names4_fullname_idx on names4 (fullname_parallel_restricted(first_name, last_name));
@@ -141,6 +167,14 @@ create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
insert into test_data1 select * from test_data where a = 10 returning a as data;
+--
+-- Test INSERT with RETURNING clause (ordered SELECT).
+-- (should create plan with parallel SELECT, GatherMerge parent node)
+--
+truncate test_data1;
+explain (costs off) insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+insert into test_data1 select * from test_data where a <= 5 order by a returning a as data;
+
--
-- Test INSERT into a table with a foreign key.
-- (Insert into a table with a foreign key is parallel-restricted,
@@ -152,6 +186,29 @@ insert into para_insert_f1 select unique1, stringu1 from tenk1;
-- select some values to verify that the insert worked
select count(*), sum(unique1) from para_insert_f1;
+--
+-- Test INSERT with underlying query, leader participation disabled
+--
+set parallel_leader_participation = off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+--
+-- Test INSERT with underlying query, leader participation disabled
+-- and no workers available
+set max_parallel_workers=0;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 2490 order by unique1;
+
+reset parallel_leader_participation;
+reset max_parallel_workers;
+
--
-- Test INSERT with ON CONFLICT ... DO UPDATE ...
-- (should not create a parallel plan)
@@ -161,6 +218,70 @@ explain (costs off) insert into test_conflict_table(id, somedata) select a, a fr
insert into test_conflict_table(id, somedata) select a, a from test_data;
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data ON CONFLICT(id) DO UPDATE SET somedata = EXCLUDED.somedata + 1;
+--
+-- Test INSERT with parallelized aggregate
+--
+create table tenk1_avg_data(count int, avg_unique1 int, avg_stringu1_len int);
+explain (costs off) insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+insert into tenk1_avg_data select count(*), avg(unique1), avg(length(stringu1)) from tenk1;
+select * from tenk1_avg_data;
+
+--
+-- Test INSERT with parallel bitmap heap scan
+--
+set enable_seqscan to off;
+set enable_indexscan to off;
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
+-- select some values to verify that the insert worked
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+reset enable_seqscan;
+reset enable_indexscan;
+
+--
+-- Test INSERT with parallel append
+--
+create table a_star_data(aa int);
+explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
+insert into a_star_data select aa from a_star where aa > 10;
+select count(aa), sum(aa) from a_star_data;
+
+--
+-- Test INSERT with parallel index scan
+--
+set enable_seqscan to off;
+set enable_bitmapscan to off;
+set min_parallel_index_scan_size=0;
+
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select * from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+--
+-- Test INSERT with parallel index-only scan
+--
+truncate para_insert_p1 cascade;
+explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
+-- select some values to verify that the parallel insert worked
+select count(*), sum(unique1) from para_insert_p1;
+select unique1 from para_insert_p1 where unique1 >= 9990 order by unique1;
+
+reset min_parallel_index_scan_size;
+reset enable_seqscan;
+reset enable_bitmapscan;
+
+--
+-- Test INSERT with parallel-safe index expression
+-- (should create a parallel plan)
+--
+explain (costs off) insert into names3 select * from names;
+insert into names3 select * from names;
+select * from names3 order by fullname_parallel_safe(first_name, last_name);
--
-- Test INSERT with parallel-unsafe index expression
@@ -197,6 +318,31 @@ create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
+--
+-- Test INSERT with parallel-safe index predicate
+-- (should create a parallel plan)
+--
+create table names8 (like names);
+create index names8_lastname_partial_idx on names8(index, last_name) where lastname_startswithe_s(last_name);
+explain (costs off) insert into names8 select * from names;
+insert into names8 select * from names;
+
+--
+-- Test INSERT with parallel-unsafe index predicate
+-- (should not create a parallel plan)
+--
+create table names9 (like names);
+create index names9_lastname_partial_idx on names9(index, last_name) where lastname_startswithe_u(last_name);
+explain (costs off) insert into names9 select * from names;
+
+--
+-- Test INSERT with parallel-restricted index predicate
+-- (should create a parallel plan)
+--
+create table names10 (like names);
+create index names10_lastname_partial_idx on names10(index, last_name) where lastname_startswithe_r(last_name);
+explain (costs off) insert into names10 select * from names;
+insert into names10 select * from names;
--
-- Test INSERT into temporary table with underlying query.
@@ -211,6 +357,19 @@ insert into temp_names select * from names;
-- Test INSERT with column defaults
--
--
+-- a: no default
+-- b: unsafe default
+-- c: restricted default
+-- d: safe default
+--
+
+--
+-- No column defaults, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
+select * from testdef order by a;
+truncate testdef;
--
-- Parallel unsafe column default, should not use a parallel plan
@@ -225,6 +384,14 @@ insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
select * from testdef order by a;
truncate testdef;
+--
+-- Parallel safe column default, should use parallel SELECT
+--
+explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
+select * from testdef order by a;
+truncate testdef;
+
--
-- Parallel restricted and unsafe column defaults, should not use a parallel plan
--
@@ -257,6 +424,46 @@ insert into parttable1 select unique1,stringu1 from tenk1;
select count(*) from parttable1_1;
select count(*) from parttable1_2;
+--
+-- Test INSERT into partition with parallel-unsafe partition key support function
+-- (should not create a parallel plan)
+--
+create function my_int4_sort(int4,int4) returns int language sql
+ as $$ select case when $1 = $2 then 0 when $1 > $2 then 1 else -1 end; $$;
+
+create operator class test_int4_ops for type int4 using btree as
+ operator 1 < (int4,int4), operator 2 <= (int4,int4),
+ operator 3 = (int4,int4), operator 4 >= (int4,int4),
+ operator 5 > (int4,int4), function 1 my_int4_sort(int4,int4);
+
+create table partkey_unsafe_key_supp_fn_t (a int4, b name) partition by range (a test_int4_ops);
+create table partkey_unsafe_key_supp_fn_t_1 partition of partkey_unsafe_key_supp_fn_t for values from (0) to (5000);
+create table partkey_unsafe_key_supp_fn_t_2 partition of partkey_unsafe_key_supp_fn_t for values from (5000) to (10000);
+
+explain (costs off) insert into partkey_unsafe_key_supp_fn_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into partition with parallel-unsafe partition key expression
+-- (should not create a parallel plan)
+--
+create table partkey_unsafe_key_expr_t (a int4, b name) partition by range ((fullname_parallel_unsafe('',a::varchar)));
+explain (costs off) insert into partkey_unsafe_key_expr_t select unique1, stringu1 from tenk1;
+
+--
+-- Test INSERT into table with parallel-safe check constraint
+-- (should create a parallel plan)
+--
+create or replace function check_a(a int4) returns boolean as $$
+ begin
+ return (a >= 0 and a <= 9999);
+ end;
+$$ language plpgsql parallel safe;
+
+create table table_check_a(a int4 check (check_a(a)), b name);
+explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
+insert into table_check_a select unique1, stringu1 from tenk1;
+select count(*), sum(a) from table_check_a;
+
--
-- Test INSERT into table with parallel-unsafe check constraint
-- (should not create a parallel plan)
@@ -271,37 +478,78 @@ create table table_check_b(a int4, b name check (check_b_unsafe(b)), c name);
explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, stringu1 from tenk1;
--
--- Test INSERT into table with parallel-safe after stmt-level triggers
+-- Test INSERT into table with parallel-safe before+after stmt-level triggers
-- (should create a parallel SELECT plan; triggers should fire)
--
create table names_with_safe_trigger (like names);
+create or replace function insert_before_trigger_safe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_safe';
+ return new;
+ end;
+$$ language plpgsql parallel safe;
create or replace function insert_after_trigger_safe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_safe';
return new;
end;
$$ language plpgsql parallel safe;
+create trigger insert_before_trigger_safe before insert on names_with_safe_trigger
+ for each statement execute procedure insert_before_trigger_safe();
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
insert into names_with_safe_trigger select * from names;
--
--- Test INSERT into table with parallel-unsafe after stmt-level triggers
+-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
-- (should not create a parallel plan; triggers should fire)
--
create table names_with_unsafe_trigger (like names);
+create or replace function insert_before_trigger_unsafe() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_unsafe';
+ return new;
+ end;
+$$ language plpgsql parallel unsafe;
create or replace function insert_after_trigger_unsafe() returns trigger as $$
begin
raise notice 'hello from insert_after_trigger_unsafe';
return new;
end;
$$ language plpgsql parallel unsafe;
+create trigger insert_before_trigger_unsafe before insert on names_with_unsafe_trigger
+ for each statement execute procedure insert_before_trigger_unsafe();
create trigger insert_after_trigger_unsafe after insert on names_with_unsafe_trigger
for each statement execute procedure insert_after_trigger_unsafe();
explain (costs off) insert into names_with_unsafe_trigger select * from names;
insert into names_with_unsafe_trigger select * from names;
+--
+-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
+-- (should create a parallel plan with parallel SELECT;
+-- stmt-level before+after triggers should fire)
+--
+create table names_with_restricted_trigger (like names);
+create or replace function insert_before_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_before_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create or replace function insert_after_trigger_restricted() returns trigger as $$
+ begin
+ raise notice 'hello from insert_after_trigger_restricted';
+ return new;
+ end;
+$$ language plpgsql parallel restricted;
+create trigger insert_before_trigger_restricted before insert on names_with_restricted_trigger
+ for each statement execute procedure insert_before_trigger_restricted();
+create trigger insert_after_trigger_restricted after insert on names_with_restricted_trigger
+ for each statement execute procedure insert_after_trigger_restricted();
+explain (costs off) insert into names_with_restricted_trigger select * from names;
+insert into names_with_restricted_trigger select * from names;
+
--
-- Test INSERT into partition with parallel-unsafe trigger
-- (should not create a parallel plan)
@@ -335,6 +583,17 @@ function make_table_bar();
-- should create a non-parallel plan
explain (costs off) execute q;
+--
+-- Test INSERT into table with TOAST column
+--
+create table insert_toast_table(index int4, data text);
+create table insert_toast_table_data (like insert_toast_table);
+insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from generate_series(1,20) as i;
+explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
+insert into insert_toast_table select index, data from insert_toast_table_data;
+select count(*) as row_count, sum(length(data)) as total_data_length from insert_toast_table;
+truncate insert_toast_table;
+
--
-- Test INSERT into table having a DOMAIN column with a CHECK constraint
--
@@ -342,15 +601,48 @@ create function sql_is_distinct_from_u(anyelement, anyelement)
returns boolean language sql parallel unsafe
as 'select $1 is distinct from $2 limit 1';
+create or replace function sql_is_distinct_from_r(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel restricted;
+
+create or replace function sql_is_distinct_from_s(a anyelement, b anyelement) returns boolean as $$
+ begin
+ return (a <> b);
+ end;
+$$ language plpgsql parallel safe;
+
create domain inotnull_u int
check (sql_is_distinct_from_u(value, null));
+create domain inotnull_r int
+ check (sql_is_distinct_from_r(value, null));
+
+create domain inotnull_s int
+ check (sql_is_distinct_from_s(value, null));
+
create table dom_table_u (x inotnull_u, y int);
+create table dom_table_r (x inotnull_r, y int);
+create table dom_table_s (x inotnull_s, y int);
-- Test INSERT into table having a DOMAIN column with parallel-unsafe CHECK constraint
explain (costs off) insert into dom_table_u select unique1, unique2 from tenk1;
+-- Test INSERT into table having a DOMAIN column with parallel-restricted CHECK constraint
+explain (costs off) insert into dom_table_r select unique1, unique2 from tenk1;
+insert into dom_table_r select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_r;
+
+-- Test INSERT into table having a DOMAIN column with parallel-safe CHECK constraint
+-- NOTE: Currently max_parallel_hazard() regards CoerceToDomain as parallel-restricted
+explain (costs off) insert into dom_table_s select unique1, unique2 from tenk1;
+insert into dom_table_s select unique1, unique2 from tenk1;
+select count(*), sum(x) as sum_x, sum(y) as sum_y from dom_table_s;
+
+
+
rollback;
@@ -361,6 +653,8 @@ rollback;
drop table names;
drop index names2_fullname_idx;
drop table names2;
+drop index names3_fullname_idx;
+drop table names3;
drop index names4_fullname_idx;
drop table names4;
drop table testdef;
@@ -370,4 +664,8 @@ drop function bdefault_unsafe;
drop function cdefault_restricted;
drop function ddefault_safe;
drop function fullname_parallel_unsafe;
+drop function fullname_parallel_safe;
drop function fullname_parallel_restricted;
+drop function lastname_startswithe_u;
+drop function lastname_startswithe_s;
+drop function lastname_startswithe_r;
--
2.27.0
v30-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchapplication/octet-stream; name=v30-0002-Enable-parallel-INSERT-and-or-SELECT-for-INSERT-INTO.patchDownload
From 24dfb8cc107bad6cc00e4b94cea8dba3d55746b7 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Mon, 22 Mar 2021 20:18:51 +1100
Subject: [PATCH 2/3] Enable parallel INSERT and/or SELECT for "INSERT INTO ...
SELECT ...", where it is safe to do so.
Parallel INSERT can't be utilized in the following cases:
- A parallel query plan can't be generated for the underlying SELECT, because,
for example, the SELECT statement uses a parallel-unsafe function
- INSERT statement uses the ON CONFLICT DO UPDATE clause
- Target table is a foreign or temporary table
- Target table has a foreign key, or a parallel-unsafe trigger, index
expression, column default expression or check constraint
- Target table is a partitioned table with a parallel-unsafe partition key
expression or support function
Where the above-mentioned target table features are found to be, at worst,
parallel-restricted, rather than parallel-unsafe, at least parallel SELECT may
be utilized for the INSERT's query plan.
Some prior work (85f6b49 and 3ba59cc) has already been done to establish the
necessary infrastructure to allow parallel INSERTs, in general, to be safe,
except for cases where new commandIds would be generated in the parallel-worker
code (such as inserts into a table having a foreign key) - these cases need to
be avoided.
The planner is updated with additional changes that build upon those made to
support parallel SELECT for "INSERT INTO ... SELECT ...". Where Gather paths are
normally generated for parallel SELECT, in the case of an
"INSERT INTO ... SELECT ...", these Gather paths are now generated only if the
parallel-safety level is found to be RESTRICTED (resulting in non-parallel
INSERT with parallel SELECT), otherwise if the parallel-safety level is found to
be parallel SAFE, then partial paths for parallel INSERT are generated before
Gather paths are added (resulting in parallel INSERT+SELECT).
The executor is updated for Gather and ModifyTable node processing, to handle
parallel INSERT, by only starting tuple queue readers if there is a RETURNING
clause, and by firing any before/after statement triggers in the leader (and
preventing them from being fired in the workers).
The handling of the currentCommandId is updated to set it as used in the leader
prior to entering parallel-mode for parallel INSERT, and to record it as used at
the start of the parallel INSERT operation in the worker.
The parallel-worker framework is updated to support serialization of an INSERT
planned statement, to be passed to the workers, and to support return of the
number of tuples processed (INSERTed) by the workers, for the executor state.
Note that this commit changes the RELATION_IS_LOCAL() macro (typically used to
decide whether we can skip acquiring locks), as now a relation created in the
current transaction can no longer be assumed to be accessible only to the
current backend, as it may be accessible to parallel workers.
Discussion: https://postgr.es/m/CAJcOf-cXnB5cnMKqWEp2E2z7Mvcd04iLVmV=qpFJrR3AcrTS3g@mail.gmail.com
---
src/backend/access/heap/heapam.c | 30 ++-
src/backend/access/transam/xact.c | 43 +++-
src/backend/executor/execMain.c | 19 +-
src/backend/executor/execParallel.c | 61 +++++-
src/backend/executor/nodeGather.c | 69 +++++-
src/backend/executor/nodeModifyTable.c | 44 +++-
src/backend/optimizer/path/costsize.c | 46 ++++
src/backend/optimizer/plan/createplan.c | 2 +-
src/backend/optimizer/plan/planner.c | 269 ++++++++++++++++--------
src/backend/optimizer/plan/setrefs.c | 23 ++
src/backend/optimizer/util/pathnode.c | 56 ++---
src/backend/storage/lmgr/README | 18 +-
src/include/access/xact.h | 3 +-
src/include/executor/execParallel.h | 1 +
src/include/executor/nodeModifyTable.h | 3 +-
src/include/nodes/execnodes.h | 3 +-
src/include/optimizer/cost.h | 1 +
src/include/optimizer/pathnode.h | 3 +-
src/include/utils/rel.h | 9 +-
19 files changed, 526 insertions(+), 177 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 7cb87f4a3b..76121a8c95 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -52,6 +52,9 @@
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/catalog.h"
+#ifdef USE_ASSERT_CHECKING
+#include "commands/trigger.h"
+#endif
#include "miscadmin.h"
#include "pgstat.h"
#include "port/atomics.h"
@@ -2294,10 +2297,31 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
* inserts in general except for the cases where inserts generate a new
* CommandId (eg. inserts into a table having a foreign key column).
*/
+#ifdef USE_ASSERT_CHECKING
if (IsParallelWorker())
- ereport(ERROR,
- (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
- errmsg("cannot insert tuples in a parallel worker")));
+ {
+ /*
+ * Assert that for this relation, no trigger of type RI_TRIGGER_FK
+ * exists, as it would indicate that the relation has a FK column,
+ * which would, on insert, result in creation of a new CommandId,
+ * and this isn't currently supported in a parallel worker.
+ */
+ TriggerDesc *trigdesc = relation->trigdesc;
+ if (trigdesc != NULL)
+ {
+ int i;
+
+ for (i = 0; i < trigdesc->numtriggers; i++)
+ {
+ int trigtype;
+ Trigger *trigger = &trigdesc->triggers[i];
+
+ trigtype = RI_FKey_trigger_type(trigger->tgfoid);
+ Assert(trigtype != RI_TRIGGER_FK);
+ }
+ }
+ }
+#endif
tup->t_data->t_infomask &= ~(HEAP_XACT_MASK);
tup->t_data->t_infomask2 &= ~(HEAP2_XACT_MASK);
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 6395a9b240..9997e07e06 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -516,6 +516,20 @@ GetCurrentFullTransactionIdIfAny(void)
return CurrentTransactionState->fullTransactionId;
}
+/*
+ * SetCurrentCommandIdUsedForWorker
+ *
+ * For a parallel worker, record that the currentCommandId has been used.
+ * This must only be called at the start of a parallel operation.
+ */
+void
+SetCurrentCommandIdUsedForWorker(void)
+{
+ Assert(IsParallelWorker() && !currentCommandIdUsed && currentCommandId != InvalidCommandId);
+
+ currentCommandIdUsed = true;
+}
+
/*
* MarkCurrentTransactionIdLoggedIfAny
*
@@ -764,12 +778,16 @@ GetCurrentCommandId(bool used)
if (used)
{
/*
- * Forbid setting currentCommandIdUsed in a parallel worker, because
- * we have no provision for communicating this back to the leader. We
- * could relax this restriction when currentCommandIdUsed was already
- * true at the start of the parallel operation.
+ * If in a parallel worker, only allow setting currentCommandIdUsed if
+ * currentCommandIdUsed was already true at the start of the parallel
+ * operation (by way of SetCurrentCommandIdUsedForWorker()), otherwise
+ * forbid setting currentCommandIdUsed because we have no provision for
+ * communicating this back to the leader. Once currentCommandIdUsed is
+ * set, the commandId used by leader and workers can't be changed,
+ * because CommandCounterIncrement() then prevents any attempted
+ * increment of the current commandId.
*/
- Assert(!IsParallelWorker());
+ Assert(!(IsParallelWorker() && !currentCommandIdUsed));
currentCommandIdUsed = true;
}
return currentCommandId;
@@ -1020,12 +1038,25 @@ IsInParallelMode(void)
* Prepare for entering parallel mode plan execution, based on command-type.
*/
void
-PrepareParallelModePlanExec(CmdType commandType)
+PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader)
{
if (IsModifySupportedInParallelMode(commandType))
{
Assert(!IsInParallelMode());
+ if (isParallelModifyLeader)
+ {
+ /*
+ * Set currentCommandIdUsed to true, to ensure that the current
+ * CommandId (which will be used by the parallel workers) won't
+ * change during this parallel operation, as starting new
+ * commands in parallel-mode is not currently supported.
+ * See related comments in GetCurrentCommandId and
+ * CommandCounterIncrement.
+ */
+ (void) GetCurrentCommandId(true);
+ }
+
/*
* Prepare for entering parallel mode by assigning a TransactionId.
* Failure to do this now would result in heap_insert() subsequently
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 0648dd82ba..0f27af6d30 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -148,12 +148,11 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
*
- * Don't allow writes in parallel mode. Supporting UPDATE and DELETE
- * would require (a) storing the combocid hash in shared memory, rather
- * than synchronizing it just once at the start of parallelism, and (b) an
- * alternative to heap_update()'s reliance on xmax for mutual exclusion.
- * INSERT may have no such troubles, but we forbid it to simplify the
- * checks.
+ * Except for INSERT, don't allow writes in parallel mode. Supporting
+ * UPDATE and DELETE would require (a) storing the combocid hash in shared
+ * memory, rather than synchronizing it just once at the start of
+ * parallelism, and (b) an alternative to heap_update()'s reliance on xmax
+ * for mutual exclusion.
*
* We have lower-level defenses in CommandCounterIncrement and elsewhere
* against performing unsafe operations in parallel mode, but this gives a
@@ -776,7 +775,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandName((Node *) plannedstmt));
}
- if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ if ((plannedstmt->commandType != CMD_SELECT &&
+ !IsModifySupportedInParallelMode(plannedstmt->commandType)) || plannedstmt->hasModifyingCTE)
PreventCommandIfParallelMode(CreateCommandName((Node *) plannedstmt));
}
@@ -1513,7 +1513,10 @@ ExecutePlan(EState *estate,
estate->es_use_parallel_mode = use_parallel_mode;
if (use_parallel_mode)
{
- PrepareParallelModePlanExec(estate->es_plannedstmt->commandType);
+ bool isParallelModifyLeader = IsA(planstate, GatherState) &&
+ IsA(outerPlanState(planstate), ModifyTableState);
+
+ PrepareParallelModePlanExec(estate->es_plannedstmt->commandType, isParallelModifyLeader);
EnterParallelMode();
}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index c95d5170e4..4a667752a1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -23,6 +23,7 @@
#include "postgres.h"
+#include "access/xact.h"
#include "executor/execParallel.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
@@ -65,6 +66,7 @@
#define PARALLEL_KEY_QUERY_TEXT UINT64CONST(0xE000000000000008)
#define PARALLEL_KEY_JIT_INSTRUMENTATION UINT64CONST(0xE000000000000009)
#define PARALLEL_KEY_WAL_USAGE UINT64CONST(0xE00000000000000A)
+#define PARALLEL_KEY_PROCESSED_COUNT UINT64CONST(0xE00000000000000B)
#define PARALLEL_TUPLE_QUEUE_SIZE 65536
@@ -173,9 +175,11 @@ ExecSerializePlan(Plan *plan, EState *estate)
* PlannedStmt to start the executor.
*/
pstmt = makeNode(PlannedStmt);
- pstmt->commandType = CMD_SELECT;
+ Assert(estate->es_plannedstmt->commandType == CMD_SELECT ||
+ IsModifySupportedInParallelMode(estate->es_plannedstmt->commandType));
+ pstmt->commandType = IsA(plan, ModifyTable) ? castNode(ModifyTable, plan)->operation : CMD_SELECT;
pstmt->queryId = UINT64CONST(0);
- pstmt->hasReturning = false;
+ pstmt->hasReturning = estate->es_plannedstmt->hasReturning;
pstmt->hasModifyingCTE = false;
pstmt->canSetTag = true;
pstmt->transientPlan = false;
@@ -183,7 +187,7 @@ ExecSerializePlan(Plan *plan, EState *estate)
pstmt->parallelModeNeeded = false;
pstmt->planTree = plan;
pstmt->rtable = estate->es_range_table;
- pstmt->resultRelations = NIL;
+ pstmt->resultRelations = estate->es_plannedstmt->resultRelations;
pstmt->appendRelations = NIL;
/*
@@ -675,6 +679,14 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
mul_size(PARALLEL_TUPLE_QUEUE_SIZE, pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /* Estimate space for returned "# of tuples processed" count. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
/*
* Give parallel-aware nodes a chance to add to the estimates, and get a
* count of how many PlanState nodes there are.
@@ -764,6 +776,19 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate,
/* We don't need the TupleQueueReaders yet, though. */
pei->reader = NULL;
+ if (IsA(planstate->plan, ModifyTable))
+ {
+ /*
+ * Allocate space for each worker's returned "# of tuples processed"
+ * count.
+ */
+ pei->processed_count = shm_toc_allocate(pcxt->toc,
+ mul_size(sizeof(uint64), pcxt->nworkers));
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PROCESSED_COUNT, pei->processed_count);
+ }
+ else
+ pei->processed_count = NULL;
+
/*
* If instrumentation options were supplied, allocate space for the data.
* It only gets partially initialized here; the rest happens during
@@ -1152,6 +1177,15 @@ ExecParallelFinish(ParallelExecutorInfo *pei)
for (i = 0; i < nworkers; i++)
InstrAccumParallelQuery(&pei->buffer_usage[i], &pei->wal_usage[i]);
+ /*
+ * Update total # of tuples processed, using counts from each worker.
+ */
+ if (pei->processed_count != NULL)
+ {
+ for (i = 0; i < nworkers; i++)
+ pei->planstate->state->es_processed += pei->processed_count[i];
+ }
+
pei->finished = true;
}
@@ -1379,6 +1413,7 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
FixedParallelExecutorState *fpes;
BufferUsage *buffer_usage;
WalUsage *wal_usage;
+ uint64 *processed_count;
DestReceiver *receiver;
QueryDesc *queryDesc;
SharedExecutorInstrumentation *instrumentation;
@@ -1400,6 +1435,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
true);
queryDesc = ExecParallelGetQueryDesc(toc, receiver, instrument_options);
+ Assert(queryDesc->operation == CMD_SELECT || IsModifySupportedInParallelMode(queryDesc->operation));
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Record that the CurrentCommandId is used, at the start of the
+ * parallel operation.
+ */
+ SetCurrentCommandIdUsedForWorker();
+ }
+
/* Setting debug_query_string for individual workers */
debug_query_string = queryDesc->sourceText;
@@ -1458,6 +1503,16 @@ ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
InstrEndParallelQuery(&buffer_usage[ParallelWorkerNumber],
&wal_usage[ParallelWorkerNumber]);
+ if (IsModifySupportedInParallelMode(queryDesc->operation))
+ {
+ /*
+ * Report the # of tuples processed during execution of a parallel
+ * table-modification command.
+ */
+ processed_count = shm_toc_lookup(toc, PARALLEL_KEY_PROCESSED_COUNT, false);
+ processed_count[ParallelWorkerNumber] = queryDesc->estate->es_processed;
+ }
+
/* Report instrumentation data if any instrumentation options are set. */
if (instrumentation != NULL)
ExecParallelReportInstrumentation(queryDesc->planstate,
diff --git a/src/backend/executor/nodeGather.c b/src/backend/executor/nodeGather.c
index 9e1dc464cb..814c0e5c93 100644
--- a/src/backend/executor/nodeGather.c
+++ b/src/backend/executor/nodeGather.c
@@ -35,6 +35,7 @@
#include "executor/execdebug.h"
#include "executor/execParallel.h"
#include "executor/nodeGather.h"
+#include "executor/nodeModifyTable.h"
#include "executor/nodeSubplan.h"
#include "executor/tqueue.h"
#include "miscadmin.h"
@@ -60,6 +61,7 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
GatherState *gatherstate;
Plan *outerNode;
TupleDesc tupDesc;
+ Index varno;
/* Gather node doesn't have innerPlan node. */
Assert(innerPlan(node) == NULL);
@@ -104,7 +106,9 @@ ExecInitGather(Gather *node, EState *estate, int eflags)
* Initialize result type and projection.
*/
ExecInitResultTypeTL(&gatherstate->ps);
- ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, OUTER_VAR);
+ varno = (IsA(outerNode, ModifyTable) && castNode(ModifyTable, outerNode)->returningLists != NULL) ?
+ castNode(ModifyTableState, outerPlanState(gatherstate))->resultRelInfo->ri_RangeTableIndex : OUTER_VAR;
+ ExecConditionalAssignProjectionInfo(&gatherstate->ps, tupDesc, varno);
/*
* Without projections result slot type is not trivially known, see
@@ -144,9 +148,19 @@ ExecGather(PlanState *pstate)
GatherState *node = castNode(GatherState, pstate);
TupleTableSlot *slot;
ExprContext *econtext;
+ ModifyTableState *nodeModifyTableState = NULL;
+ bool isModify = false;
+ bool isModifyWithReturning = false;
CHECK_FOR_INTERRUPTS();
+ if (IsA(outerPlanState(pstate), ModifyTableState))
+ {
+ nodeModifyTableState = castNode(ModifyTableState, outerPlanState(pstate));
+ isModify = IsModifySupportedInParallelMode(nodeModifyTableState->operation);
+ isModifyWithReturning = isModify && nodeModifyTableState->ps.plan->targetlist != NIL;
+ }
+
/*
* Initialize the parallel context and workers on first execution. We do
* this on first execution rather than during node initialization, as it
@@ -178,6 +192,16 @@ ExecGather(PlanState *pstate)
node->pei,
gather->initParam);
+ if (isModify)
+ {
+ /*
+ * For a supported parallel table-modification command, if
+ * there are BEFORE STATEMENT triggers, these must be fired by
+ * the leader, not by the parallel workers.
+ */
+ fireBSTriggersInLeader(nodeModifyTableState);
+ }
+
/*
* Register backend workers. We might not get as many as we
* requested, or indeed any at all.
@@ -188,7 +212,7 @@ ExecGather(PlanState *pstate)
node->nworkers_launched = pcxt->nworkers_launched;
/* Set up tuple queue readers to read the results. */
- if (pcxt->nworkers_launched > 0)
+ if (pcxt->nworkers_launched > 0 && (!isModify || isModifyWithReturning))
{
ExecParallelCreateReaders(node->pei);
/* Make a working array showing the active readers */
@@ -200,7 +224,11 @@ ExecGather(PlanState *pstate)
}
else
{
- /* No workers? Then never mind. */
+ /*
+ * No workers were launched, or this is a supported parallel
+ * table-modification command without a RETURNING clause - no
+ * readers are required.
+ */
node->nreaders = 0;
node->reader = NULL;
}
@@ -208,7 +236,7 @@ ExecGather(PlanState *pstate)
}
/* Run plan locally if no workers or enabled and not single-copy. */
- node->need_to_scan_locally = (node->nreaders == 0)
+ node->need_to_scan_locally = (node->nworkers_launched <= 0)
|| (!gather->single_copy && parallel_leader_participation);
node->initialized = true;
}
@@ -229,7 +257,7 @@ ExecGather(PlanState *pstate)
return NULL;
/* If no projection is required, we're done. */
- if (node->ps.ps_ProjInfo == NULL)
+ if (node->ps.ps_ProjInfo == NULL || isModifyWithReturning)
return slot;
/*
@@ -418,14 +446,35 @@ ExecShutdownGatherWorkers(GatherState *node)
void
ExecShutdownGather(GatherState *node)
{
- ExecShutdownGatherWorkers(node);
+ bool isModify;
- /* Now destroy the parallel context. */
- if (node->pei != NULL)
+ /*
+ * If the parallel context has already been destroyed, this function must
+ * have been previously called, so just return.
+ */
+ if (node->pei == NULL)
+ return;
+
+ isModify = IsA(outerPlanState(node), ModifyTableState) &&
+ IsModifySupportedInParallelMode(castNode(ModifyTableState, outerPlanState(node))->operation);
+
+ if (isModify)
{
- ExecParallelCleanup(node->pei);
- node->pei = NULL;
+ /*
+ * For a supported parallel table-modification command, if there are
+ * AFTER STATEMENT triggers, these must be fired by the leader, not by
+ * the parallel workers.
+ */
+ ModifyTableState *nodeModifyTableState = castNode(ModifyTableState, outerPlanState(node));
+
+ fireASTriggersInLeader(nodeModifyTableState);
}
+
+ ExecShutdownGatherWorkers(node);
+
+ /* Now destroy the parallel context. */
+ ExecParallelCleanup(node->pei);
+ node->pei = NULL;
}
/* ----------------------------------------------------------------
diff --git a/src/backend/executor/nodeModifyTable.c b/src/backend/executor/nodeModifyTable.c
index 2993ba43e3..3ff85599cb 100644
--- a/src/backend/executor/nodeModifyTable.c
+++ b/src/backend/executor/nodeModifyTable.c
@@ -39,6 +39,7 @@
#include "access/heapam.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/tableam.h"
#include "access/xact.h"
#include "catalog/catalog.h"
@@ -1950,6 +1951,39 @@ fireASTriggers(ModifyTableState *node)
}
}
+/*
+ * Process BEFORE EACH STATEMENT triggers, in the leader
+ */
+void
+fireBSTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ if (node->fireBSTriggers)
+ {
+ fireBSTriggers(node);
+ node->fireBSTriggers = false;
+
+ /*
+ * Disable firing of AFTER STATEMENT triggers by local plan execution
+ * (ModifyTable processing). These will be fired at the end of Gather
+ * processing.
+ */
+ node->fireASTriggers = false;
+ }
+}
+
+/*
+ * Process AFTER EACH STATEMENT triggers, in the leader
+ */
+void
+fireASTriggersInLeader(ModifyTableState *node)
+{
+ Assert(IsInParallelMode() && !IsParallelWorker());
+
+ fireASTriggers(node);
+}
+
/*
* Set up the state needed for collecting transition tuples for AFTER
* triggers.
@@ -2298,7 +2332,11 @@ ExecModifyTable(PlanState *pstate)
/*
* We're done, but fire AFTER STATEMENT triggers before exiting.
*/
- fireASTriggers(node);
+ if (node->fireASTriggers)
+ {
+ fireASTriggers(node);
+ node->fireASTriggers = false;
+ }
node->mt_done = true;
@@ -2375,7 +2413,9 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
/* set up epqstate with dummy subplan data for the moment */
EvalPlanQualInit(&mtstate->mt_epqstate, estate, NULL, NIL, node->epqParam);
- mtstate->fireBSTriggers = true;
+ /* Statement-level triggers must not be fired by parallel workers */
+ mtstate->fireBSTriggers = !IsParallelWorker();
+ mtstate->fireASTriggers = !IsParallelWorker();
/*
* Build state for collecting transition tuples. This requires having a
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c81e2cf244..7618737ea2 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -214,6 +214,52 @@ clamp_row_est(double nrows)
}
+/*
+ * cost_modifytable
+ * Determines and returns the cost of a ModifyTable node.
+ */
+void
+cost_modifytable(ModifyTablePath *path)
+{
+ double total_size;
+ ListCell *lc;
+
+ /*
+ * Compute cost & rowcount as sum of subpath costs & rowcounts.
+ *
+ * Currently, we don't charge anything extra for the actual table
+ * modification work, nor for the WITH CHECK OPTIONS or RETURNING
+ * expressions if any.
+ */
+ path->path.startup_cost = 0;
+ path->path.total_cost = 0;
+ path->path.rows = 0;
+ total_size = 0;
+ foreach(lc, path->subpaths)
+ {
+ Path *subpath = (Path *) lfirst(lc);
+
+ if (lc == list_head(path->subpaths)) /* first node? */
+ path->path.startup_cost = subpath->startup_cost;
+ path->path.total_cost += subpath->total_cost;
+ if (path->returningLists != NIL)
+ {
+ path->path.rows += subpath->rows;
+ total_size += subpath->pathtarget->width * subpath->rows;
+ }
+ }
+
+ /*
+ * Set width to the average width of the subpath outputs. XXX this is
+ * totally wrong: we should return an average of the RETURNING tlist
+ * widths. But it's what happened historically, and improving it is a
+ * task for another day.
+ */
+ if (path->path.rows > 0)
+ total_size /= path->path.rows;
+ path->path.pathtarget->width = rint(total_size);
+}
+
/*
* cost_seqscan
* Determines and returns the cost of scanning a relation sequentially.
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 906cab7053..2a3ad42f90 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -345,7 +345,7 @@ create_plan(PlannerInfo *root, Path *best_path)
* top-level tlist seen at execution time. However, ModifyTable plan
* nodes don't have a tlist matching the querytree targetlist.
*/
- if (!IsA(plan, ModifyTable))
+ if (!IsA(plan, ModifyTable) && !(IsA(plan, Gather) && IsA(outerPlan(plan), ModifyTable)))
apply_tlist_labeling(plan->targetlist, root->processed_tlist);
/*
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 28b40dd905..1abfaebb76 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -130,6 +130,9 @@ typedef struct
static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
static void inheritance_planner(PlannerInfo *root);
+static Path *generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path, int64 offset_est,
+ int64 count_est, bool isParallelModify);
static void grouping_planner(PlannerInfo *root, bool inheritance_update,
double tuple_fraction);
static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
@@ -323,10 +326,11 @@ standard_planner(Query *parse, const char *query_string, int cursorOptions,
*
* (Note that we do allow CREATE TABLE AS, INSERT INTO...SELECT, SELECT
* INTO, and CREATE MATERIALIZED VIEW to use parallel plans. However, as
- * of now, only the leader backend writes into a completely new table. In
- * the future, we can extend it to allow workers to write into the table.
- * However, to allow parallel updates and deletes, we have to solve other
- * problems, especially around combo CIDs.)
+ * of now, only INSERT INTO...SELECT employs workers to write into the
+ * table, while for the other cases only the leader backend writes into a
+ * completely new table. In the future, we can extend it to allow workers
+ * for more cases. However, to allow parallel updates and deletes, we have
+ * to solve other problems, especially around combo CIDs.)
*
* For now, we don't try to use parallel mode if we're running inside a
* parallel worker. We might eventually be able to relax this
@@ -1807,7 +1811,120 @@ inheritance_planner(PlannerInfo *root)
returningLists,
rowMarks,
NULL,
- assign_special_exec_param(root)));
+ assign_special_exec_param(root),
+ 0));
+}
+
+/*
+ * generate_final_rel_path
+ * Generate a path for the final_rel, with LockRows, Limit, and/or
+ * ModifyTable steps added if needed.
+ */
+static Path *
+generate_final_rel_path(PlannerInfo *root, RelOptInfo *final_rel,
+ bool inheritance_update, Path *path,
+ int64 offset_est, int64 count_est, bool isParallelModify)
+{
+ Query *parse = root->parse;
+
+ /*
+ * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
+ * (Note: we intentionally test parse->rowMarks not root->rowMarks
+ * here. If there are only non-locking rowmarks, they should be
+ * handled by the ModifyTable node instead. However, root->rowMarks
+ * is what goes into the LockRows node.)
+ */
+ if (parse->rowMarks)
+ {
+ path = (Path *) create_lockrows_path(root, final_rel, path,
+ root->rowMarks,
+ assign_special_exec_param(root));
+ }
+
+ /*
+ * If there is a LIMIT/OFFSET clause, add the LIMIT node.
+ */
+ if (limit_needed(parse))
+ {
+ path = (Path *) create_limit_path(root, final_rel, path,
+ parse->limitOffset,
+ parse->limitCount,
+ parse->limitOption,
+ offset_est, count_est);
+ }
+
+ /*
+ * If this is an INSERT/UPDATE/DELETE, and we're not being called from
+ * inheritance_planner, add the ModifyTable node.
+ */
+ if (parse->commandType != CMD_SELECT && !inheritance_update)
+ {
+ Index rootRelation;
+ List *withCheckOptionLists;
+ List *returningLists;
+ List *rowMarks;
+ int parallelWorkers;
+
+ /*
+ * If target is a partition root table, we need to mark the
+ * ModifyTable node appropriately for that.
+ */
+ if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
+ RELKIND_PARTITIONED_TABLE)
+ rootRelation = parse->resultRelation;
+ else
+ rootRelation = 0;
+
+ /*
+ * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
+ * needed.
+ */
+ if (parse->withCheckOptions)
+ withCheckOptionLists = list_make1(parse->withCheckOptions);
+ else
+ withCheckOptionLists = NIL;
+
+ if (parse->returningList)
+ returningLists = list_make1(parse->returningList);
+ else
+ returningLists = NIL;
+
+ /*
+ * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
+ * will have dealt with fetching non-locked marked rows, else we
+ * need to have ModifyTable do that.
+ */
+ if (parse->rowMarks)
+ rowMarks = NIL;
+ else
+ rowMarks = root->rowMarks;
+
+ /*
+ * For the number of workers to use for a parallel
+ * INSERT/UPDATE/DELETE, it seems reasonable to use the same number
+ * of workers as estimated for the underlying query.
+ */
+ parallelWorkers = isParallelModify ? path->parallel_workers : 0;
+
+ path = (Path *)
+ create_modifytable_path(root, final_rel,
+ parse->commandType,
+ parse->canSetTag,
+ parse->resultRelation,
+ rootRelation,
+ false,
+ list_make1_int(parse->resultRelation),
+ list_make1(path),
+ list_make1(root),
+ withCheckOptionLists,
+ returningLists,
+ rowMarks,
+ parse->onConflict,
+ assign_special_exec_param(root),
+ parallelWorkers);
+ }
+
+ return path;
}
/*--------------------
@@ -1855,6 +1972,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
RelOptInfo *final_rel;
FinalPathExtraData extra;
ListCell *lc;
+ bool parallel_modify_partial_path_added = false;
/* Tweak caller-supplied tuple_fraction if have LIMIT/OFFSET */
if (parse->limitCount || parse->limitOffset)
@@ -2295,96 +2413,33 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
{
Path *path = (Path *) lfirst(lc);
- /*
- * If there is a FOR [KEY] UPDATE/SHARE clause, add the LockRows node.
- * (Note: we intentionally test parse->rowMarks not root->rowMarks
- * here. If there are only non-locking rowmarks, they should be
- * handled by the ModifyTable node instead. However, root->rowMarks
- * is what goes into the LockRows node.)
- */
- if (parse->rowMarks)
- {
- path = (Path *) create_lockrows_path(root, final_rel, path,
- root->rowMarks,
- assign_special_exec_param(root));
- }
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, false);
- /*
- * If there is a LIMIT/OFFSET clause, add the LIMIT node.
- */
- if (limit_needed(parse))
- {
- path = (Path *) create_limit_path(root, final_rel, path,
- parse->limitOffset,
- parse->limitCount,
- parse->limitOption,
- offset_est, count_est);
- }
+ /* And shove it into final_rel */
+ add_path(final_rel, path);
+ }
+ /* Consider a supported parallel table-modification command */
+ if (IsModifySupportedInParallelMode(parse->commandType) &&
+ !inheritance_update &&
+ final_rel->consider_parallel &&
+ parse->rowMarks == NIL)
+ {
/*
- * If this is an INSERT/UPDATE/DELETE, and we're not being called from
- * inheritance_planner, add the ModifyTable node.
+ * Generate partial paths for the final_rel. Insert all surviving
+ * paths, with Limit, and/or ModifyTable steps added if needed.
*/
- if (parse->commandType != CMD_SELECT && !inheritance_update)
+ foreach(lc, current_rel->partial_pathlist)
{
- Index rootRelation;
- List *withCheckOptionLists;
- List *returningLists;
- List *rowMarks;
-
- /*
- * If target is a partition root table, we need to mark the
- * ModifyTable node appropriately for that.
- */
- if (rt_fetch(parse->resultRelation, parse->rtable)->relkind ==
- RELKIND_PARTITIONED_TABLE)
- rootRelation = parse->resultRelation;
- else
- rootRelation = 0;
-
- /*
- * Set up the WITH CHECK OPTION and RETURNING lists-of-lists, if
- * needed.
- */
- if (parse->withCheckOptions)
- withCheckOptionLists = list_make1(parse->withCheckOptions);
- else
- withCheckOptionLists = NIL;
-
- if (parse->returningList)
- returningLists = list_make1(parse->returningList);
- else
- returningLists = NIL;
+ Path *path = (Path *) lfirst(lc);
- /*
- * If there was a FOR [KEY] UPDATE/SHARE clause, the LockRows node
- * will have dealt with fetching non-locked marked rows, else we
- * need to have ModifyTable do that.
- */
- if (parse->rowMarks)
- rowMarks = NIL;
- else
- rowMarks = root->rowMarks;
+ path = generate_final_rel_path(root, final_rel, inheritance_update, path,
+ offset_est, count_est, true);
- path = (Path *)
- create_modifytable_path(root, final_rel,
- parse->commandType,
- parse->canSetTag,
- parse->resultRelation,
- rootRelation,
- false,
- list_make1_int(parse->resultRelation),
- list_make1(path),
- list_make1(root),
- withCheckOptionLists,
- returningLists,
- rowMarks,
- parse->onConflict,
- assign_special_exec_param(root));
+ add_partial_path(final_rel, path);
+ parallel_modify_partial_path_added = true;
}
-
- /* And shove it into final_rel */
- add_path(final_rel, path);
}
/*
@@ -2403,6 +2458,18 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
}
}
+ if (parallel_modify_partial_path_added)
+ {
+ /*
+ * Generate gather paths according to the added partial paths for the
+ * parallel table-modification command.
+ * Note that true is passed for the "override_rows" parameter, so that
+ * the rows from the cheapest partial path (ModifyTablePath) are used,
+ * not the rel's (possibly estimated) rows.
+ */
+ generate_useful_gather_paths(root, final_rel, true);
+ }
+
extra.limit_needed = limit_needed(parse);
extra.limit_tuples = limit_tuples;
extra.count_est = count_est;
@@ -7572,7 +7639,33 @@ apply_scanjoin_target_to_paths(PlannerInfo *root,
* one of the generated paths may turn out to be the cheapest one.
*/
if (rel->consider_parallel && !IS_OTHER_REL(rel))
- generate_useful_gather_paths(root, rel, false);
+ {
+ if (IsModifySupportedInParallelMode(root->parse->commandType))
+ {
+ Assert(root->glob->parallelModeOK);
+ if (root->glob->maxParallelHazard != PROPARALLEL_SAFE)
+ {
+ /*
+ * Don't allow a supported parallel table-modification
+ * command, because it's not safe.
+ */
+ if (root->glob->maxParallelHazard == PROPARALLEL_RESTRICTED)
+ {
+ /*
+ * However, do allow any underlying query to be run by
+ * parallel workers.
+ */
+ generate_useful_gather_paths(root, rel, false);
+ }
+ rel->partial_pathlist = NIL;
+ rel->consider_parallel = false;
+ }
+ }
+ else
+ {
+ generate_useful_gather_paths(root, rel, false);
+ }
+ }
/*
* Reassess which paths are the cheapest, now that we've potentially added
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 42f088ad71..4d7773538e 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -1066,6 +1066,29 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
plan->lefttree = set_plan_refs(root, plan->lefttree, rtoffset);
plan->righttree = set_plan_refs(root, plan->righttree, rtoffset);
+ /*
+ * FIXME: The following code block is a bit of a hack to fix the missing
+ * targetlist on the Gather node, in the case of an underlying ModifyTable
+ * node for Parallel INSERT. The current design expects the ModifyTable
+ * targetlist to be set in set_plan_refs(), but the targetlist is needed
+ * by the parent Gather node, which is processed first.
+ * This issue is a consequence of the fact that, prior to Parallel INSERT
+ * support, ModifyTable node was always a top-level plan node. Now the
+ * ModifyTable node may be in the subplan of the Gather node, so the
+ * expected order of node processing and configuration has changed.
+ * Currently it is not known how to fix this issue in a more elegant way.
+ */
+ if (nodeTag(plan) == T_Gather)
+ {
+ Plan *subplan = plan->lefttree;
+
+ if (IsA(subplan, ModifyTable) &&
+ castNode(ModifyTable, subplan)->returningLists != NIL)
+ {
+ plan->targetlist = subplan->targetlist;
+ }
+ }
+
return plan;
}
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 69b83071cf..e6af7d3067 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -3553,6 +3553,7 @@ create_lockrows_path(PlannerInfo *root, RelOptInfo *rel,
* 'rowMarks' is a list of PlanRowMarks (non-locking only)
* 'onconflict' is the ON CONFLICT clause, or NULL
* 'epqParam' is the ID of Param for EvalPlanQual re-eval
+ * 'parallelWorkers' is the no. of parallel workers to use
*/
ModifyTablePath *
create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
@@ -3563,10 +3564,10 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam)
+ int epqParam,
+ int parallelWorkers)
{
ModifyTablePath *pathnode = makeNode(ModifyTablePath);
- double total_size;
ListCell *lc;
Assert(list_length(resultRelations) == list_length(subpaths));
@@ -3583,47 +3584,22 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
/* For now, assume we are above any joins, so no parameterization */
pathnode->path.param_info = NULL;
pathnode->path.parallel_aware = false;
- pathnode->path.parallel_safe = false;
- pathnode->path.parallel_workers = 0;
- pathnode->path.pathkeys = NIL;
-
- /*
- * Compute cost & rowcount as sum of subpath costs & rowcounts.
- *
- * Currently, we don't charge anything extra for the actual table
- * modification work, nor for the WITH CHECK OPTIONS or RETURNING
- * expressions if any. It would only be window dressing, since
- * ModifyTable is always a top-level node and there is no way for the
- * costs to change any higher-level planning choices. But we might want
- * to make it look better sometime.
- */
- pathnode->path.startup_cost = 0;
- pathnode->path.total_cost = 0;
- pathnode->path.rows = 0;
- total_size = 0;
- foreach(lc, subpaths)
+ pathnode->path.parallel_safe = rel->consider_parallel && parallelWorkers > 0;
+ if (pathnode->path.parallel_safe)
{
- Path *subpath = (Path *) lfirst(lc);
-
- if (lc == list_head(subpaths)) /* first node? */
- pathnode->path.startup_cost = subpath->startup_cost;
- pathnode->path.total_cost += subpath->total_cost;
- if (returningLists != NIL)
+ foreach(lc, subpaths)
{
- pathnode->path.rows += subpath->rows;
- total_size += subpath->pathtarget->width * subpath->rows;
+ Path *sp = (Path *) lfirst(lc);
+
+ if (!sp->parallel_safe)
+ {
+ pathnode->path.parallel_safe = false;
+ break;
+ }
}
}
-
- /*
- * Set width to the average width of the subpath outputs. XXX this is
- * totally wrong: we should return an average of the RETURNING tlist
- * widths. But it's what happened historically, and improving it is a task
- * for another day.
- */
- if (pathnode->path.rows > 0)
- total_size /= pathnode->path.rows;
- pathnode->path.pathtarget->width = rint(total_size);
+ pathnode->path.parallel_workers = parallelWorkers;
+ pathnode->path.pathkeys = NIL;
pathnode->operation = operation;
pathnode->canSetTag = canSetTag;
@@ -3639,6 +3615,8 @@ create_modifytable_path(PlannerInfo *root, RelOptInfo *rel,
pathnode->onconflict = onconflict;
pathnode->epqParam = epqParam;
+ cost_modifytable(pathnode);
+
return pathnode;
}
diff --git a/src/backend/storage/lmgr/README b/src/backend/storage/lmgr/README
index c96cc7b7c5..9e75cff7de 100644
--- a/src/backend/storage/lmgr/README
+++ b/src/backend/storage/lmgr/README
@@ -633,12 +633,13 @@ operation at the same time which would ordinarily be prevented by the
heavyweight lock mechanism, undefined behavior might result. In practice, the
dangers are modest. The leader and worker share the same transaction,
snapshot, and combo CID hash, and neither can perform any DDL or, indeed,
-write any data at all. Thus, for either to read a table locked exclusively by
-the other is safe enough. Problems would occur if the leader initiated
-parallelism from a point in the code at which it had some backend-private
-state that made table access from another process unsafe, for example after
-calling SetReindexProcessing and before calling ResetReindexProcessing,
-catastrophe could ensue, because the worker won't have that state.
+write any data at all (with the exception of parallel insert). Thus, for either
+to read a table locked exclusively by the other is safe enough. Problems would
+occur if the leader initiated parallelism from a point in the code at which it
+had some backend-private state that made table access from another process
+unsafe, for example after calling SetReindexProcessing and before calling
+ResetReindexProcessing, catastrophe could ensue, because the worker won't have
+that state.
To allow parallel inserts and parallel copy, we have ensured that relation
extension and page locks don't participate in group locking which means such
@@ -652,9 +653,8 @@ happens, so those will also not participate in deadlock. To allow for other
parallel writes like parallel update or parallel delete, we'll either need to
(1) further enhance the deadlock detector to handle those tuple locks in a
different way than other types; or (2) have parallel workers use some other
-mutual exclusion method for such cases. Currently, the parallel mode is
-strictly read-only, but now we have the infrastructure to allow parallel
-inserts and parallel copy.
+mutual exclusion method for such cases. Currently, only parallel insert is
+allowed, but we have the infrastructure to allow parallel copy.
Group locking adds three new members to each PGPROC: lockGroupLeader,
lockGroupMembers, and lockGroupLink. A PGPROC's lockGroupLeader is NULL for
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 34cfaf542c..ea814e48fc 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -386,6 +386,7 @@ extern FullTransactionId GetTopFullTransactionId(void);
extern FullTransactionId GetTopFullTransactionIdIfAny(void);
extern FullTransactionId GetCurrentFullTransactionId(void);
extern FullTransactionId GetCurrentFullTransactionIdIfAny(void);
+extern void SetCurrentCommandIdUsedForWorker(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
@@ -466,7 +467,7 @@ extern void ParsePrepareRecord(uint8 info, xl_xact_prepare *xlrec, xl_xact_parse
extern void EnterParallelMode(void);
extern void ExitParallelMode(void);
extern bool IsInParallelMode(void);
-extern void PrepareParallelModePlanExec(CmdType commandType);
+extern void PrepareParallelModePlanExec(CmdType commandType, bool isParallelModifyLeader);
/*
* IsModifySupportedInParallelMode
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 3888175a2f..072869fdda 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -27,6 +27,7 @@ typedef struct ParallelExecutorInfo
ParallelContext *pcxt; /* parallel context we're using */
BufferUsage *buffer_usage; /* points to bufusage area in DSM */
WalUsage *wal_usage; /* walusage area in DSM */
+ uint64 *processed_count; /* processed tuple count area in DSM */
SharedExecutorInstrumentation *instrumentation; /* optional */
struct SharedJitInstrumentation *jit_instrumentation; /* optional */
dsa_area *area; /* points to DSA area in DSM */
diff --git a/src/include/executor/nodeModifyTable.h b/src/include/executor/nodeModifyTable.h
index 83e2965531..5206a4e6fd 100644
--- a/src/include/executor/nodeModifyTable.h
+++ b/src/include/executor/nodeModifyTable.h
@@ -22,5 +22,6 @@ extern void ExecComputeStoredGenerated(ResultRelInfo *resultRelInfo,
extern ModifyTableState *ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags);
extern void ExecEndModifyTable(ModifyTableState *node);
extern void ExecReScanModifyTable(ModifyTableState *node);
-
+extern void fireBSTriggersInLeader(ModifyTableState *node);
+extern void fireASTriggersInLeader(ModifyTableState *node);
#endif /* NODEMODIFYTABLE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index e31ad6204e..27272ec2c8 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1177,7 +1177,8 @@ typedef struct ModifyTableState
List **mt_arowmarks; /* per-subplan ExecAuxRowMark lists */
EPQState mt_epqstate; /* for evaluating EvalPlanQual rechecks */
- bool fireBSTriggers; /* do we need to fire stmt triggers? */
+ bool fireBSTriggers; /* do we need to fire before stmt triggers? */
+ bool fireASTriggers; /* do we need to fire after stmt triggers? */
/*
* Slot for storing tuples in the root partitioned table's rowtype during
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 22e6db96b6..a6a53599cd 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -70,6 +70,7 @@ extern PGDLLIMPORT int constraint_exclusion;
extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
+extern void cost_modifytable(ModifyTablePath *path);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 54f4b782fc..cb36413e5b 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -267,7 +267,8 @@ extern ModifyTablePath *create_modifytable_path(PlannerInfo *root,
List *subroots,
List *withCheckOptionLists, List *returningLists,
List *rowMarks, OnConflictExpr *onconflict,
- int epqParam);
+ int epqParam,
+ int parallel_workers);
extern LimitPath *create_limit_path(PlannerInfo *root, RelOptInfo *rel,
Path *subpath,
Node *limitOffset, Node *limitCount,
diff --git a/src/include/utils/rel.h b/src/include/utils/rel.h
index 5375a37dd1..9b8d459e9c 100644
--- a/src/include/utils/rel.h
+++ b/src/include/utils/rel.h
@@ -15,6 +15,7 @@
#define REL_H
#include "access/tupdesc.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/pg_class.h"
#include "catalog/pg_index.h"
@@ -600,15 +601,15 @@ typedef struct PartitionedTableRdOptions
/*
* RELATION_IS_LOCAL
- * If a rel is either temp or newly created in the current transaction,
- * it can be assumed to be accessible only to the current backend.
- * This is typically used to decide that we can skip acquiring locks.
+ * If a rel is temp, it can be assumed to be accessible only to the
+ * current backend. This is typically used to decide that we can
+ * skip acquiring locks.
*
* Beware of multiple eval of argument
*/
#define RELATION_IS_LOCAL(relation) \
((relation)->rd_islocaltemp || \
- (relation)->rd_createSubid != InvalidSubTransactionId)
+ (!IsInParallelMode() && ((relation)->rd_createSubid != InvalidSubTransactionId)))
/*
* RELATION_IS_OTHER_TEMP
--
2.27.0
v30-0003-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchapplication/octet-stream; name=v30-0003-Parallel-INSERT-and-or-SELECT-for-INSERT-INTO-tests-and-doc.patchDownload
From f4d048269ae8d4f048a318ae1f872b7fe1a920a8 Mon Sep 17 00:00:00 2001
From: Greg Nancarrow <gregn4422@gmail.com>
Date: Thu, 18 Mar 2021 15:18:35 +1100
Subject: [PATCH 3/3] INSERT and/or SELECT for "INSERT INTO ... SELECT ..." -
tests and documentation updates.
---
doc/src/sgml/parallel.sgml | 81 +++++++++-
src/test/regress/expected/insert_parallel.out | 146 +++++++++---------
src/test/regress/sql/insert_parallel.sql | 28 ++--
3 files changed, 164 insertions(+), 91 deletions(-)
diff --git a/doc/src/sgml/parallel.sgml b/doc/src/sgml/parallel.sgml
index 330c06b608..ffdbd91701 100644
--- a/doc/src/sgml/parallel.sgml
+++ b/doc/src/sgml/parallel.sgml
@@ -141,12 +141,16 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
<itemizedlist>
<listitem>
<para>
- The query writes any data or locks any database rows. If a query
- contains a data-modifying operation either at the top level or within
- a CTE, no parallel plans for that query will be generated. As an
- exception, the following commands which create a new table and populate
- it can use a parallel plan for the underlying <literal>SELECT</literal>
- part of the query:
+ The query locks any database rows, or writes data within a CTE or using
+ a parallel-unsupported data-modifying operation. Currently, the only
+ parallel-supported data-modifying operations are
+ <literal>INSERT INTO ... SELECT ...</literal>, and the table creation
+ and population commands <literal>CREATE TABLE ... AS</literal>,
+ <literal>SELECT INTO</literal>, and
+ <literal>CREATE MATERIALIZED VIEW</literal>. If a query contains a
+ parallel-unsupported data-modifying operation at the top level, or any
+ data-modifying operation within a CTE, no parallel plans for that query
+ will be generated.
<itemizedlist>
<listitem>
@@ -442,6 +446,71 @@ EXPLAIN SELECT * FROM pgbench_accounts WHERE filler LIKE '%x%';
</para>
</sect2>
+ <sect2 id="parallel-insert">
+ <title>Parallel Insert</title>
+
+ <para>
+ When an <literal>INSERT</literal> statement uses an underlying
+ <literal>SELECT</literal> query to supply the rows to be inserted, a
+ parallel query plan may be generated for which the work of both data
+ retrieval and data insertion is divided amongst the workers. In this
+ case, each worker handles insertion of its portion of the rows retrieved
+ by the underlying <literal>SELECT</literal> query.
+ </para>
+
+ <para>
+ Parallel <literal>INSERT</literal> is not supported in all situations. The
+ operations invoked by the <literal>INSERT</literal> statement must be
+ parallel-safe, including those that are invoked as a result of certain
+ features of the target table.
+ </para>
+
+ <para>
+ For example, any of the following will prevent the use of parallel
+ <literal>INSERT</literal> in the query plan:
+ </para>
+
+ <itemizedlist>
+ <listitem>
+ <para>
+ A parallel query plan can't be generated for the underlying
+ <literal>SELECT</literal>, because, for example, the
+ <literal>SELECT</literal> statement uses a parallel-unsafe function.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The INSERT statement uses an ON CONFLICT DO UPDATE clause.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a foreign or temporary table.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table has a foreign key, or has a parallel-unsafe trigger,
+ index expression, column default expression or check constraint.
+ </para>
+ </listitem>
+ <listitem>
+ <para>
+ The target table is a partitioned table with a parallel-unsafe partition
+ key expression or support function.
+ </para>
+ </listitem>
+ </itemizedlist>
+
+ <para>
+ Where the target table is a foreign or temporary table, or the above target
+ table features are determined to be, at worst, parallel-restricted, rather
+ than parallel-unsafe, at least a parallel table scan may be used in the
+ query plan for the <literal>INSERT</literal> statement. For more information
+ about Parallel Safety, see <xref linkend="parallel-safety"/>.
+ </para>
+ </sect2>
+
<sect2 id="parallel-plan-tips">
<title>Parallel Plan Tips</title>
diff --git a/src/test/regress/expected/insert_parallel.out b/src/test/regress/expected/insert_parallel.out
index 2fef07bb6c..2c0c42b11c 100644
--- a/src/test/regress/expected/insert_parallel.out
+++ b/src/test/regress/expected/insert_parallel.out
@@ -123,14 +123,14 @@ explain(costs off) insert into para_insert_p1 select unique1, stringu1 from tenk
alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -151,7 +151,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
NOTICE: truncate cascades to table "para_insert_f1"
@@ -183,15 +183,15 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
QUERY PLAN
--------------------------------------------
- Insert on test_data1
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_data1
-> Parallel Seq Scan on test_data
Filter: (a = 10)
(5 rows)
@@ -261,9 +261,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -300,9 +300,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 <= 2500;
QUERY PLAN
-----------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Seq Scan on tenk1
Filter: (unique1 <= 2500)
(5 rows)
@@ -340,9 +340,9 @@ create table test_conflict_table(id serial primary key, somedata int);
explain (costs off) insert into test_conflict_table(id, somedata) select a, a from test_data;
QUERY PLAN
--------------------------------------------
- Insert on test_conflict_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on test_conflict_table
-> Parallel Seq Scan on test_data
(4 rows)
@@ -389,9 +389,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 7500;
QUERY PLAN
------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Bitmap Heap Scan on tenk1
Recheck Cond: (unique1 >= 7500)
-> Bitmap Index Scan on tenk1_unique1
@@ -424,9 +424,9 @@ create table a_star_data(aa int);
explain (costs off) insert into a_star_data select aa from a_star where aa > 10;
QUERY PLAN
--------------------------------------------------------
- Insert on a_star_data
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on a_star_data
-> Parallel Append
-> Parallel Seq Scan on d_star a_star_4
Filter: (aa > 10)
@@ -460,9 +460,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 where unique1 >= 500;
QUERY PLAN
--------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -498,9 +498,9 @@ NOTICE: truncate cascades to table "para_insert_f1"
explain (costs off) insert into para_insert_p1 select unique1 from tenk1 where unique1 >= 500;
QUERY PLAN
-------------------------------------------------------------------
- Insert on para_insert_p1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on para_insert_p1
-> Parallel Index Only Scan using tenk1_unique1 on tenk1
Index Cond: (unique1 >= 500)
(5 rows)
@@ -538,9 +538,9 @@ reset enable_bitmapscan;
explain (costs off) insert into names3 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names3
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names3
-> Parallel Seq Scan on names
(4 rows)
@@ -584,21 +584,21 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
QUERY PLAN
----------------------------------------
- Insert on names5
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names5
-> Parallel Seq Scan on names
(4 rows)
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -627,7 +627,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -663,9 +663,9 @@ create index names8_lastname_partial_idx on names8(index, last_name) where lastn
explain (costs off) insert into names8 select * from names;
QUERY PLAN
----------------------------------------
- Insert on names8
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on names8
-> Parallel Seq Scan on names
(4 rows)
@@ -725,14 +725,14 @@ insert into temp_names select * from names;
-- d: safe default
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -764,7 +764,7 @@ explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
(2 rows)
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
QUERY PLAN
@@ -793,14 +793,14 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
QUERY PLAN
--------------------------------------------
- Insert on testdef
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on testdef
-> Parallel Seq Scan on test_data
(4 rows)
@@ -859,9 +859,9 @@ alter table parttable1 set (parallel_insert_enabled = on);
explain (costs off) insert into parttable1 select unique1,stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on parttable1
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on parttable1
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -923,9 +923,9 @@ create table table_check_a(a int4 check (check_a(a)), b name);
explain (costs off) insert into table_check_a select unique1, stringu1 from tenk1;
QUERY PLAN
----------------------------------------
- Insert on table_check_a
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on table_check_a
-> Parallel Seq Scan on tenk1
(4 rows)
@@ -955,7 +955,8 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan; triggers should fire)
+-- (should create a parallel INSERT+SELECT plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
create or replace function insert_before_trigger_safe() returns trigger as $$
@@ -975,11 +976,11 @@ create trigger insert_before_trigger_safe before insert on names_with_safe_trigg
create trigger insert_after_trigger_safe after insert on names_with_safe_trigger
for each statement execute procedure insert_after_trigger_safe();
explain (costs off) insert into names_with_safe_trigger select * from names;
- QUERY PLAN
-----------------------------------------
- Insert on names_with_safe_trigger
- -> Gather
- Workers Planned: 3
+ QUERY PLAN
+-----------------------------------------
+ Gather
+ Workers Planned: 3
+ -> Insert on names_with_safe_trigger
-> Parallel Seq Scan on names
(4 rows)
@@ -988,7 +989,8 @@ NOTICE: hello from insert_before_trigger_safe
NOTICE: hello from insert_after_trigger_safe
--
-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
--- (should not create a parallel plan; triggers should fire)
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
create or replace function insert_before_trigger_unsafe() returns trigger as $$
@@ -1019,7 +1021,7 @@ NOTICE: hello from insert_before_trigger_unsafe
NOTICE: hello from insert_after_trigger_unsafe
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -1076,13 +1078,13 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
QUERY PLAN
--------------------------------------
- Insert on rp
- -> Gather
- Workers Planned: 4
+ Gather
+ Workers Planned: 4
+ -> Insert on rp
-> Parallel Seq Scan on foo
Filter: ((a % 2) = 0)
(5 rows)
@@ -1109,9 +1111,9 @@ insert into insert_toast_table_data select i, rpad('T', 16384, 'ABCDEFGH') from
explain (costs off) insert into insert_toast_table select index, data from insert_toast_table_data;
QUERY PLAN
----------------------------------------------------------
- Insert on insert_toast_table
- -> Gather
- Workers Planned: 3
+ Gather
+ Workers Planned: 3
+ -> Insert on insert_toast_table
-> Parallel Seq Scan on insert_toast_table_data
(4 rows)
diff --git a/src/test/regress/sql/insert_parallel.sql b/src/test/regress/sql/insert_parallel.sql
index cd8f9ef00e..5cb96c8c60 100644
--- a/src/test/regress/sql/insert_parallel.sql
+++ b/src/test/regress/sql/insert_parallel.sql
@@ -138,7 +138,7 @@ alter table para_insert_p1 set (parallel_insert_enabled = on);
--
-- Test INSERT with underlying query.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1;
insert into para_insert_p1 select unique1, stringu1 from tenk1;
@@ -149,7 +149,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with ordered underlying query.
--- (should create plan with parallel SELECT, GatherMerge parent node)
+-- (should create plan with INSERT + parallel SELECT, GatherMerge parent node)
--
truncate para_insert_p1 cascade;
explain (costs off) insert into para_insert_p1 select unique1, stringu1 from tenk1 order by unique1;
@@ -161,7 +161,7 @@ select count(*) from (select distinct cmin,xmin from para_insert_p1) as dt;
--
-- Test INSERT with RETURNING clause.
--- (should create plan with parallel SELECT, Gather parent node)
+-- (should create plan with parallel INSERT+SELECT, Gather parent node)
--
create table test_data1(like test_data);
explain (costs off) insert into test_data1 select * from test_data where a = 10 returning a as data;
@@ -297,14 +297,14 @@ explain (costs off) insert into names4 select * from names;
--
-- Test INSERT with underlying query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; parallel INSERT+SELECT)
--
create table names5 (like names);
explain (costs off) insert into names5 select * from names returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (no projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names6 (like names);
explain (costs off) insert into names6 select * from names order by last_name returning *;
@@ -312,7 +312,7 @@ insert into names6 select * from names order by last_name returning *;
--
-- Test INSERT with underlying ordered query - and RETURNING (with projection)
--- (should create a parallel plan; parallel SELECT)
+-- (should create a parallel plan; INSERT + parallel SELECT)
--
create table names7 (like names);
explain (costs off) insert into names7 select * from names order by last_name returning last_name || ', ' || first_name as last_name_then_first_name;
@@ -364,7 +364,7 @@ insert into temp_names select * from names;
--
--
--- No column defaults, should use parallel SELECT
+-- No column defaults, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
insert into testdef(a,b,c,d) select a,a*2,a*4,a*8 from test_data;
@@ -377,7 +377,7 @@ truncate testdef;
explain (costs off) insert into testdef(a,c,d) select a,a*4,a*8 from test_data;
--
--- Parallel restricted column default, should use parallel SELECT
+-- Parallel restricted column default, should use INSERT + parallel SELECT
--
explain (costs off) insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
insert into testdef(a,b,d) select a,a*2,a*8 from test_data;
@@ -385,7 +385,7 @@ select * from testdef order by a;
truncate testdef;
--
--- Parallel safe column default, should use parallel SELECT
+-- Parallel safe column default, should use parallel INSERT+SELECT
--
explain (costs off) insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
insert into testdef(a,b,c) select a,a*2,a*4 from test_data;
@@ -479,7 +479,8 @@ explain (costs off) insert into table_check_b(a,b,c) select unique1, unique2, st
--
-- Test INSERT into table with parallel-safe before+after stmt-level triggers
--- (should create a parallel SELECT plan; triggers should fire)
+-- (should create a parallel INSERT+SELECT plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_safe_trigger (like names);
create or replace function insert_before_trigger_safe() returns trigger as $$
@@ -503,7 +504,8 @@ insert into names_with_safe_trigger select * from names;
--
-- Test INSERT into table with parallel-unsafe before+after stmt-level triggers
--- (should not create a parallel plan; triggers should fire)
+-- (should not create a parallel plan;
+-- stmt-level before+after triggers should fire)
--
create table names_with_unsafe_trigger (like names);
create or replace function insert_before_trigger_unsafe() returns trigger as $$
@@ -527,7 +529,7 @@ insert into names_with_unsafe_trigger select * from names;
--
-- Test INSERT into table with parallel-restricted before+after stmt-level trigger
--- (should create a parallel plan with parallel SELECT;
+-- (should create a parallel plan with INSERT + parallel SELECT;
-- stmt-level before+after triggers should fire)
--
create table names_with_restricted_trigger (like names);
@@ -573,7 +575,7 @@ create table rp1 partition of rp for values from (minvalue) to (0);
create table rp2 partition of rp for values from (0) to (maxvalue);
create table foo (a) as select unique1 from tenk1;
prepare q as insert into rp select * from foo where a%2 = 0;
--- should create a parallel plan
+-- should create a plan with parallel INSERT+SELECT
explain (costs off) execute q;
create or replace function make_table_bar () returns trigger language
--
2.27.0
On Mon, Mar 22, 2021 at 3:57 PM Greg Nancarrow <gregn4422@gmail.com> wrote:
On Mon, Mar 22, 2021 at 6:28 PM houzj.fnst@fujitsu.com
<houzj.fnst@fujitsu.com> wrote:Let me know if these changes seem OK to you.
Yes, these changes look good to me.
Posting an updated set of patches with these changes...
I have marked this as Returned with Feedback. There is a lot of work
to do for this patch as per the feedback given on pgsql-committers
[1]: /messages/by-id/E1lMiB9-0001c3-SY@gemulon.postgresql.org
[1]: /messages/by-id/E1lMiB9-0001c3-SY@gemulon.postgresql.org
--
With Regards,
Amit Kapila.