1
0
mirror of https://github.com/postgres/postgres.git synced 2025-10-25 13:17:41 +03:00

Handle INSERT .. ON CONFLICT with partitioned tables

Commit eb7ed3f306 enabled unique constraints on partitioned tables,
but one thing that was not working properly is INSERT/ON CONFLICT.
This commit introduces a new node keeps state related to the ON CONFLICT
clause per partition, and fills it when that partition is about to be
used for tuple routing.

Author: Amit Langote, Álvaro Herrera
Reviewed-by: Etsuro Fujita, Pavan Deolasee
Discussion: https://postgr.es/m/20180228004602.cwdyralmg5ejdqkq@alvherre.pgsql
This commit is contained in:
Alvaro Herrera
2018-03-26 10:43:54 -03:00
parent 1b89c2188b
commit 555ee77a96
14 changed files with 638 additions and 84 deletions

View File

@@ -3322,21 +3322,6 @@ ALTER TABLE measurement ATTACH PARTITION measurement_y2008m02
</para> </para>
</listitem> </listitem>
<listitem>
<para>
Using the <literal>ON CONFLICT</literal> clause with partitioned tables
will cause an error if the conflict target is specified (see
<xref linkend="sql-on-conflict" /> for more details on how the clause
works). Therefore, it is not possible to specify
<literal>DO UPDATE</literal> as the alternative action, because
specifying the conflict target is mandatory in that case. On the other
hand, specifying <literal>DO NOTHING</literal> as the alternative action
works fine provided the conflict target is not specified. In that case,
unique constraints (or exclusion constraints) of the individual leaf
partitions are considered.
</para>
</listitem>
<listitem> <listitem>
<para> <para>
When an <command>UPDATE</command> causes a row to move from one When an <command>UPDATE</command> causes a row to move from one

View File

@@ -518,6 +518,14 @@ INSERT INTO <replaceable class="parameter">table_name</replaceable> [ AS <replac
not duplicate each other in terms of attributes constrained by an not duplicate each other in terms of attributes constrained by an
arbiter index or constraint. arbiter index or constraint.
</para> </para>
<para>
Note that it is currently not supported for the
<literal>ON CONFLICT DO UPDATE</literal> clause of an
<command>INSERT</command> applied to a partitioned table to update the
partition key of a conflicting row such that it requires the row be moved
to a new partition.
</para>
<tip> <tip>
<para> <para>
It is often preferable to use unique index inference rather than It is often preferable to use unique index inference rather than

View File

@@ -138,6 +138,10 @@ typedef struct PartitionRangeBound
bool lower; /* this is the lower (vs upper) bound */ bool lower; /* this is the lower (vs upper) bound */
} PartitionRangeBound; } PartitionRangeBound;
static Oid get_partition_parent_worker(Relation inhRel, Oid relid);
static void get_partition_ancestors_worker(Relation inhRel, Oid relid,
List **ancestors);
static int32 qsort_partition_hbound_cmp(const void *a, const void *b); static int32 qsort_partition_hbound_cmp(const void *a, const void *b);
static int32 qsort_partition_list_value_cmp(const void *a, const void *b, static int32 qsort_partition_list_value_cmp(const void *a, const void *b,
void *arg); void *arg);
@@ -1377,6 +1381,7 @@ check_default_allows_bound(Relation parent, Relation default_rel,
/* /*
* get_partition_parent * get_partition_parent
* Obtain direct parent of given relation
* *
* Returns inheritance parent of a partition by scanning pg_inherits * Returns inheritance parent of a partition by scanning pg_inherits
* *
@@ -1387,15 +1392,34 @@ check_default_allows_bound(Relation parent, Relation default_rel,
Oid Oid
get_partition_parent(Oid relid) get_partition_parent(Oid relid)
{ {
Form_pg_inherits form;
Relation catalogRelation; Relation catalogRelation;
SysScanDesc scan;
ScanKeyData key[2];
HeapTuple tuple;
Oid result; Oid result;
catalogRelation = heap_open(InheritsRelationId, AccessShareLock); catalogRelation = heap_open(InheritsRelationId, AccessShareLock);
result = get_partition_parent_worker(catalogRelation, relid);
if (!OidIsValid(result))
elog(ERROR, "could not find tuple for parent of relation %u", relid);
heap_close(catalogRelation, AccessShareLock);
return result;
}
/*
* get_partition_parent_worker
* Scan the pg_inherits relation to return the OID of the parent of the
* given relation
*/
static Oid
get_partition_parent_worker(Relation inhRel, Oid relid)
{
SysScanDesc scan;
ScanKeyData key[2];
Oid result = InvalidOid;
HeapTuple tuple;
ScanKeyInit(&key[0], ScanKeyInit(&key[0],
Anum_pg_inherits_inhrelid, Anum_pg_inherits_inhrelid,
BTEqualStrategyNumber, F_OIDEQ, BTEqualStrategyNumber, F_OIDEQ,
@@ -1405,22 +1429,64 @@ get_partition_parent(Oid relid)
BTEqualStrategyNumber, F_INT4EQ, BTEqualStrategyNumber, F_INT4EQ,
Int32GetDatum(1)); Int32GetDatum(1));
scan = systable_beginscan(catalogRelation, InheritsRelidSeqnoIndexId, true, scan = systable_beginscan(inhRel, InheritsRelidSeqnoIndexId, true,
NULL, 2, key); NULL, 2, key);
tuple = systable_getnext(scan); tuple = systable_getnext(scan);
if (!HeapTupleIsValid(tuple)) if (HeapTupleIsValid(tuple))
elog(ERROR, "could not find tuple for parent of relation %u", relid); {
Form_pg_inherits form = (Form_pg_inherits) GETSTRUCT(tuple);
form = (Form_pg_inherits) GETSTRUCT(tuple); result = form->inhparent;
result = form->inhparent; }
systable_endscan(scan); systable_endscan(scan);
heap_close(catalogRelation, AccessShareLock);
return result; return result;
} }
/*
* get_partition_ancestors
* Obtain ancestors of given relation
*
* Returns a list of ancestors of the given relation.
*
* Note: Because this function assumes that the relation whose OID is passed
* as an argument and each ancestor will have precisely one parent, it should
* only be called when it is known that the relation is a partition.
*/
List *
get_partition_ancestors(Oid relid)
{
List *result = NIL;
Relation inhRel;
inhRel = heap_open(InheritsRelationId, AccessShareLock);
get_partition_ancestors_worker(inhRel, relid, &result);
heap_close(inhRel, AccessShareLock);
return result;
}
/*
* get_partition_ancestors_worker
* recursive worker for get_partition_ancestors
*/
static void
get_partition_ancestors_worker(Relation inhRel, Oid relid, List **ancestors)
{
Oid parentOid;
/* Recursion ends at the topmost level, ie., when there's no parent */
parentOid = get_partition_parent_worker(inhRel, relid);
if (parentOid == InvalidOid)
return;
*ancestors = lappend_oid(*ancestors, parentOid);
get_partition_ancestors_worker(inhRel, parentOid, ancestors);
}
/* /*
* get_qual_from_partbound * get_qual_from_partbound
* Given a parser node for partition bound, return the list of executable * Given a parser node for partition bound, return the list of executable

View File

@@ -1347,11 +1347,15 @@ InitResultRelInfo(ResultRelInfo *resultRelInfo,
resultRelInfo->ri_FdwRoutine = GetFdwRoutineForRelation(resultRelationDesc, true); resultRelInfo->ri_FdwRoutine = GetFdwRoutineForRelation(resultRelationDesc, true);
else else
resultRelInfo->ri_FdwRoutine = NULL; resultRelInfo->ri_FdwRoutine = NULL;
/* The following fields are set later if needed */
resultRelInfo->ri_FdwState = NULL; resultRelInfo->ri_FdwState = NULL;
resultRelInfo->ri_usesFdwDirectModify = false; resultRelInfo->ri_usesFdwDirectModify = false;
resultRelInfo->ri_ConstraintExprs = NULL; resultRelInfo->ri_ConstraintExprs = NULL;
resultRelInfo->ri_junkFilter = NULL; resultRelInfo->ri_junkFilter = NULL;
resultRelInfo->ri_projectReturning = NULL; resultRelInfo->ri_projectReturning = NULL;
resultRelInfo->ri_onConflictArbiterIndexes = NIL;
resultRelInfo->ri_onConflict = NULL;
/* /*
* Partition constraint, which also includes the partition constraint of * Partition constraint, which also includes the partition constraint of

View File

@@ -15,10 +15,12 @@
#include "postgres.h" #include "postgres.h"
#include "catalog/pg_inherits_fn.h" #include "catalog/pg_inherits_fn.h"
#include "catalog/pg_type.h"
#include "executor/execPartition.h" #include "executor/execPartition.h"
#include "executor/executor.h" #include "executor/executor.h"
#include "mb/pg_wchar.h" #include "mb/pg_wchar.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/rls.h" #include "utils/rls.h"
#include "utils/ruleutils.h" #include "utils/ruleutils.h"
@@ -36,6 +38,8 @@ static char *ExecBuildSlotPartitionKeyDescription(Relation rel,
Datum *values, Datum *values,
bool *isnull, bool *isnull,
int maxfieldlen); int maxfieldlen);
static List *adjust_partition_tlist(List *tlist, TupleConversionMap *map);
/* /*
* ExecSetupPartitionTupleRouting - sets up information needed during * ExecSetupPartitionTupleRouting - sets up information needed during
@@ -64,6 +68,8 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
int num_update_rri = 0, int num_update_rri = 0,
update_rri_index = 0; update_rri_index = 0;
PartitionTupleRouting *proute; PartitionTupleRouting *proute;
int nparts;
ModifyTable *node = mtstate ? (ModifyTable *) mtstate->ps.plan : NULL;
/* /*
* Get the information about the partition tree after locking all the * Get the information about the partition tree after locking all the
@@ -74,20 +80,16 @@ ExecSetupPartitionTupleRouting(ModifyTableState *mtstate, Relation rel)
proute->partition_dispatch_info = proute->partition_dispatch_info =
RelationGetPartitionDispatchInfo(rel, &proute->num_dispatch, RelationGetPartitionDispatchInfo(rel, &proute->num_dispatch,
&leaf_parts); &leaf_parts);
proute->num_partitions = list_length(leaf_parts); proute->num_partitions = nparts = list_length(leaf_parts);
proute->partitions = (ResultRelInfo **) palloc(proute->num_partitions * proute->partitions =
sizeof(ResultRelInfo *)); (ResultRelInfo **) palloc(nparts * sizeof(ResultRelInfo *));
proute->parent_child_tupconv_maps = proute->parent_child_tupconv_maps =
(TupleConversionMap **) palloc0(proute->num_partitions * (TupleConversionMap **) palloc0(nparts * sizeof(TupleConversionMap *));
sizeof(TupleConversionMap *)); proute->partition_oids = (Oid *) palloc(nparts * sizeof(Oid));
proute->partition_oids = (Oid *) palloc(proute->num_partitions *
sizeof(Oid));
/* Set up details specific to the type of tuple routing we are doing. */ /* Set up details specific to the type of tuple routing we are doing. */
if (mtstate && mtstate->operation == CMD_UPDATE) if (node && node->operation == CMD_UPDATE)
{ {
ModifyTable *node = (ModifyTable *) mtstate->ps.plan;
update_rri = mtstate->resultRelInfo; update_rri = mtstate->resultRelInfo;
num_update_rri = list_length(node->plans); num_update_rri = list_length(node->plans);
proute->subplan_partition_offsets = proute->subplan_partition_offsets =
@@ -328,7 +330,7 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
*/ */
oldContext = MemoryContextSwitchTo(estate->es_query_cxt); oldContext = MemoryContextSwitchTo(estate->es_query_cxt);
leaf_part_rri = (ResultRelInfo *) palloc0(sizeof(ResultRelInfo)); leaf_part_rri = makeNode(ResultRelInfo);
InitResultRelInfo(leaf_part_rri, InitResultRelInfo(leaf_part_rri,
partrel, partrel,
node ? node->nominalRelation : 1, node ? node->nominalRelation : 1,
@@ -475,9 +477,6 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
&mtstate->ps, RelationGetDescr(partrel)); &mtstate->ps, RelationGetDescr(partrel));
} }
Assert(proute->partitions[partidx] == NULL);
proute->partitions[partidx] = leaf_part_rri;
/* /*
* Save a tuple conversion map to convert a tuple routed to this partition * Save a tuple conversion map to convert a tuple routed to this partition
* from the parent's type to the partition's. * from the parent's type to the partition's.
@@ -487,6 +486,145 @@ ExecInitPartitionInfo(ModifyTableState *mtstate,
RelationGetDescr(partrel), RelationGetDescr(partrel),
gettext_noop("could not convert row type")); gettext_noop("could not convert row type"));
/*
* If there is an ON CONFLICT clause, initialize state for it.
*/
if (node && node->onConflictAction != ONCONFLICT_NONE)
{
TupleConversionMap *map = proute->parent_child_tupconv_maps[partidx];
int firstVarno = mtstate->resultRelInfo[0].ri_RangeTableIndex;
Relation firstResultRel = mtstate->resultRelInfo[0].ri_RelationDesc;
TupleDesc partrelDesc = RelationGetDescr(partrel);
ExprContext *econtext = mtstate->ps.ps_ExprContext;
ListCell *lc;
List *arbiterIndexes = NIL;
/*
* If there is a list of arbiter indexes, map it to a list of indexes
* in the partition. We do that by scanning the partition's index
* list and searching for ancestry relationships to each index in the
* ancestor table.
*/
if (list_length(resultRelInfo->ri_onConflictArbiterIndexes) > 0)
{
List *childIdxs;
childIdxs = RelationGetIndexList(leaf_part_rri->ri_RelationDesc);
foreach(lc, childIdxs)
{
Oid childIdx = lfirst_oid(lc);
List *ancestors;
ListCell *lc2;
ancestors = get_partition_ancestors(childIdx);
foreach(lc2, resultRelInfo->ri_onConflictArbiterIndexes)
{
if (list_member_oid(ancestors, lfirst_oid(lc2)))
arbiterIndexes = lappend_oid(arbiterIndexes, childIdx);
}
list_free(ancestors);
}
}
/*
* If the resulting lists are of inequal length, something is wrong.
* (This shouldn't happen, since arbiter index selection should not
* pick up an invalid index.)
*/
if (list_length(resultRelInfo->ri_onConflictArbiterIndexes) !=
list_length(arbiterIndexes))
elog(ERROR, "invalid arbiter index list");
leaf_part_rri->ri_onConflictArbiterIndexes = arbiterIndexes;
/*
* In the DO UPDATE case, we have some more state to initialize.
*/
if (node->onConflictAction == ONCONFLICT_UPDATE)
{
Assert(node->onConflictSet != NIL);
Assert(resultRelInfo->ri_onConflict != NULL);
/*
* If the partition's tuple descriptor matches exactly the root
* parent (the common case), we can simply re-use the parent's ON
* CONFLICT SET state, skipping a bunch of work. Otherwise, we
* need to create state specific to this partition.
*/
if (map == NULL)
leaf_part_rri->ri_onConflict = resultRelInfo->ri_onConflict;
else
{
List *onconflset;
TupleDesc tupDesc;
bool found_whole_row;
leaf_part_rri->ri_onConflict = makeNode(OnConflictSetState);
/*
* Translate expressions in onConflictSet to account for
* different attribute numbers. For that, map partition
* varattnos twice: first to catch the EXCLUDED
* pseudo-relation (INNER_VAR), and second to handle the main
* target relation (firstVarno).
*/
onconflset = (List *) copyObject((Node *) node->onConflictSet);
onconflset =
map_partition_varattnos(onconflset, INNER_VAR, partrel,
firstResultRel, &found_whole_row);
Assert(!found_whole_row);
onconflset =
map_partition_varattnos(onconflset, firstVarno, partrel,
firstResultRel, &found_whole_row);
Assert(!found_whole_row);
/* Finally, adjust this tlist to match the partition. */
onconflset = adjust_partition_tlist(onconflset, map);
/*
* Build UPDATE SET's projection info. The user of this
* projection is responsible for setting the slot's tupdesc!
* We set aside a tupdesc that's good for the common case of a
* partition that's tupdesc-equal to the partitioned table;
* partitions of different tupdescs must generate their own.
*/
tupDesc = ExecTypeFromTL(onconflset, partrelDesc->tdhasoid);
ExecSetSlotDescriptor(mtstate->mt_conflproj, tupDesc);
leaf_part_rri->ri_onConflict->oc_ProjInfo =
ExecBuildProjectionInfo(onconflset, econtext,
mtstate->mt_conflproj,
&mtstate->ps, partrelDesc);
leaf_part_rri->ri_onConflict->oc_ProjTupdesc = tupDesc;
/*
* If there is a WHERE clause, initialize state where it will
* be evaluated, mapping the attribute numbers appropriately.
* As with onConflictSet, we need to map partition varattnos
* to the partition's tupdesc.
*/
if (node->onConflictWhere)
{
List *clause;
clause = copyObject((List *) node->onConflictWhere);
clause = map_partition_varattnos(clause, INNER_VAR,
partrel, firstResultRel,
&found_whole_row);
Assert(!found_whole_row);
clause = map_partition_varattnos(clause, firstVarno,
partrel, firstResultRel,
&found_whole_row);
Assert(!found_whole_row);
leaf_part_rri->ri_onConflict->oc_WhereClause =
ExecInitQual((List *) clause, &mtstate->ps);
}
}
}
}
Assert(proute->partitions[partidx] == NULL);
proute->partitions[partidx] = leaf_part_rri;
MemoryContextSwitchTo(oldContext); MemoryContextSwitchTo(oldContext);
return leaf_part_rri; return leaf_part_rri;
@@ -946,3 +1084,70 @@ ExecBuildSlotPartitionKeyDescription(Relation rel,
return buf.data; return buf.data;
} }
/*
* adjust_partition_tlist
* Adjust the targetlist entries for a given partition to account for
* attribute differences between parent and the partition
*
* The expressions have already been fixed, but here we fix the list to make
* target resnos match the partition's attribute numbers. This results in a
* copy of the original target list in which the entries appear in resno
* order, including both the existing entries (that may have their resno
* changed in-place) and the newly added entries for columns that don't exist
* in the parent.
*
* Scribbles on the input tlist, so callers must make sure to make a copy
* before passing it to us.
*/
static List *
adjust_partition_tlist(List *tlist, TupleConversionMap *map)
{
List *new_tlist = NIL;
TupleDesc tupdesc = map->outdesc;
AttrNumber *attrMap = map->attrMap;
AttrNumber attrno;
for (attrno = 1; attrno <= tupdesc->natts; attrno++)
{
Form_pg_attribute att_tup = TupleDescAttr(tupdesc, attrno - 1);
TargetEntry *tle;
if (attrMap[attrno - 1] != InvalidAttrNumber)
{
Assert(!att_tup->attisdropped);
/*
* Use the corresponding entry from the parent's tlist, adjusting
* the resno the match the partition's attno.
*/
tle = (TargetEntry *) list_nth(tlist, attrMap[attrno - 1] - 1);
tle->resno = attrno;
}
else
{
Const *expr;
/*
* For a dropped attribute in the partition, generate a dummy
* entry with resno matching the partition's attno.
*/
Assert(att_tup->attisdropped);
expr = makeConst(INT4OID,
-1,
InvalidOid,
sizeof(int32),
(Datum) 0,
true, /* isnull */
true /* byval */ );
tle = makeTargetEntry((Expr *) expr,
attrno,
pstrdup(NameStr(att_tup->attname)),
false);
}
new_tlist = lappend(new_tlist, tle);
}
return new_tlist;
}

View File

@@ -422,7 +422,7 @@ ExecInsert(ModifyTableState *mtstate,
bool specConflict; bool specConflict;
List *arbiterIndexes; List *arbiterIndexes;
arbiterIndexes = node->arbiterIndexes; arbiterIndexes = resultRelInfo->ri_onConflictArbiterIndexes;
/* /*
* Do a non-conclusive check for conflicts first. * Do a non-conclusive check for conflicts first.
@@ -1055,6 +1055,18 @@ lreplace:;
int map_index; int map_index;
TupleConversionMap *tupconv_map; TupleConversionMap *tupconv_map;
/*
* Disallow an INSERT ON CONFLICT DO UPDATE that causes the
* original row to migrate to a different partition. Maybe this
* can be implemented some day, but it seems a fringe feature with
* little redeeming value.
*/
if (((ModifyTable *) mtstate->ps.plan)->onConflictAction == ONCONFLICT_UPDATE)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("invalid ON UPDATE specification"),
errdetail("The result tuple would appear in a different partition than the original tuple.")));
/* /*
* When an UPDATE is run on a leaf partition, we will not have * When an UPDATE is run on a leaf partition, we will not have
* partition tuple routing set up. In that case, fail with * partition tuple routing set up. In that case, fail with
@@ -1313,7 +1325,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate,
{ {
ExprContext *econtext = mtstate->ps.ps_ExprContext; ExprContext *econtext = mtstate->ps.ps_ExprContext;
Relation relation = resultRelInfo->ri_RelationDesc; Relation relation = resultRelInfo->ri_RelationDesc;
ExprState *onConflictSetWhere = resultRelInfo->ri_onConflictSetWhere; ExprState *onConflictSetWhere = resultRelInfo->ri_onConflict->oc_WhereClause;
HeapTupleData tuple; HeapTupleData tuple;
HeapUpdateFailureData hufd; HeapUpdateFailureData hufd;
LockTupleMode lockmode; LockTupleMode lockmode;
@@ -1462,7 +1474,7 @@ ExecOnConflictUpdate(ModifyTableState *mtstate,
} }
/* Project the new tuple version */ /* Project the new tuple version */
ExecProject(resultRelInfo->ri_onConflictSetProj); ExecProject(resultRelInfo->ri_onConflict->oc_ProjInfo);
/* /*
* Note that it is possible that the target tuple has been modified in * Note that it is possible that the target tuple has been modified in
@@ -1639,6 +1651,7 @@ ExecPrepareTupleRouting(ModifyTableState *mtstate,
ResultRelInfo *targetRelInfo, ResultRelInfo *targetRelInfo,
TupleTableSlot *slot) TupleTableSlot *slot)
{ {
ModifyTable *node;
int partidx; int partidx;
ResultRelInfo *partrel; ResultRelInfo *partrel;
HeapTuple tuple; HeapTuple tuple;
@@ -1720,6 +1733,19 @@ ExecPrepareTupleRouting(ModifyTableState *mtstate,
proute->partition_tuple_slot, proute->partition_tuple_slot,
&slot); &slot);
/* Initialize information needed to handle ON CONFLICT DO UPDATE. */
Assert(mtstate != NULL);
node = (ModifyTable *) mtstate->ps.plan;
if (node->onConflictAction == ONCONFLICT_UPDATE)
{
Assert(mtstate->mt_existing != NULL);
ExecSetSlotDescriptor(mtstate->mt_existing,
RelationGetDescr(partrel->ri_RelationDesc));
Assert(mtstate->mt_conflproj != NULL);
ExecSetSlotDescriptor(mtstate->mt_conflproj,
partrel->ri_onConflict->oc_ProjTupdesc);
}
return slot; return slot;
} }
@@ -2347,11 +2373,15 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
mtstate->ps.ps_ExprContext = NULL; mtstate->ps.ps_ExprContext = NULL;
} }
/* Set the list of arbiter indexes if needed for ON CONFLICT */
resultRelInfo = mtstate->resultRelInfo;
if (node->onConflictAction != ONCONFLICT_NONE)
resultRelInfo->ri_onConflictArbiterIndexes = node->arbiterIndexes;
/* /*
* If needed, Initialize target list, projection and qual for ON CONFLICT * If needed, Initialize target list, projection and qual for ON CONFLICT
* DO UPDATE. * DO UPDATE.
*/ */
resultRelInfo = mtstate->resultRelInfo;
if (node->onConflictAction == ONCONFLICT_UPDATE) if (node->onConflictAction == ONCONFLICT_UPDATE)
{ {
ExprContext *econtext; ExprContext *econtext;
@@ -2368,34 +2398,54 @@ ExecInitModifyTable(ModifyTable *node, EState *estate, int eflags)
econtext = mtstate->ps.ps_ExprContext; econtext = mtstate->ps.ps_ExprContext;
relationDesc = resultRelInfo->ri_RelationDesc->rd_att; relationDesc = resultRelInfo->ri_RelationDesc->rd_att;
/* initialize slot for the existing tuple */ /*
* Initialize slot for the existing tuple. If we'll be performing
* tuple routing, the tuple descriptor to use for this will be
* determined based on which relation the update is actually applied
* to, so we don't set its tuple descriptor here.
*/
mtstate->mt_existing = mtstate->mt_existing =
ExecInitExtraTupleSlot(mtstate->ps.state, relationDesc); ExecInitExtraTupleSlot(mtstate->ps.state,
mtstate->mt_partition_tuple_routing ?
NULL : relationDesc);
/* carried forward solely for the benefit of explain */ /* carried forward solely for the benefit of explain */
mtstate->mt_excludedtlist = node->exclRelTlist; mtstate->mt_excludedtlist = node->exclRelTlist;
/* create target slot for UPDATE SET projection */ /* create state for DO UPDATE SET operation */
resultRelInfo->ri_onConflict = makeNode(OnConflictSetState);
/*
* Create the tuple slot for the UPDATE SET projection.
*
* Just like mt_existing above, we leave it without a tuple descriptor
* in the case of partitioning tuple routing, so that it can be
* changed by ExecPrepareTupleRouting. In that case, we still save
* the tupdesc in the parent's state: it can be reused by partitions
* with an identical descriptor to the parent.
*/
tupDesc = ExecTypeFromTL((List *) node->onConflictSet, tupDesc = ExecTypeFromTL((List *) node->onConflictSet,
relationDesc->tdhasoid); relationDesc->tdhasoid);
mtstate->mt_conflproj = mtstate->mt_conflproj =
ExecInitExtraTupleSlot(mtstate->ps.state, tupDesc); ExecInitExtraTupleSlot(mtstate->ps.state,
mtstate->mt_partition_tuple_routing ?
NULL : tupDesc);
resultRelInfo->ri_onConflict->oc_ProjTupdesc = tupDesc;
/* build UPDATE SET projection state */ /* build UPDATE SET projection state */
resultRelInfo->ri_onConflictSetProj = resultRelInfo->ri_onConflict->oc_ProjInfo =
ExecBuildProjectionInfo(node->onConflictSet, econtext, ExecBuildProjectionInfo(node->onConflictSet, econtext,
mtstate->mt_conflproj, &mtstate->ps, mtstate->mt_conflproj, &mtstate->ps,
relationDesc); relationDesc);
/* build DO UPDATE WHERE clause expression */ /* initialize state to evaluate the WHERE clause, if any */
if (node->onConflictWhere) if (node->onConflictWhere)
{ {
ExprState *qualexpr; ExprState *qualexpr;
qualexpr = ExecInitQual((List *) node->onConflictWhere, qualexpr = ExecInitQual((List *) node->onConflictWhere,
&mtstate->ps); &mtstate->ps);
resultRelInfo->ri_onConflict->oc_WhereClause = qualexpr;
resultRelInfo->ri_onConflictSetWhere = qualexpr;
} }
} }

View File

@@ -1026,13 +1026,6 @@ transformOnConflictClause(ParseState *pstate,
TargetEntry *te; TargetEntry *te;
int attno; int attno;
if (targetrel->rd_partdesc)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("%s cannot be applied to partitioned table \"%s\"",
"ON CONFLICT DO UPDATE",
RelationGetRelationName(targetrel))));
/* /*
* All INSERT expressions have been parsed, get ready for potentially * All INSERT expressions have been parsed, get ready for potentially
* existing SET statements that need to be processed like an UPDATE. * existing SET statements that need to be processed like an UPDATE.

View File

@@ -52,6 +52,7 @@ extern PartitionBoundInfo partition_bounds_copy(PartitionBoundInfo src,
extern void check_new_partition_bound(char *relname, Relation parent, extern void check_new_partition_bound(char *relname, Relation parent,
PartitionBoundSpec *spec); PartitionBoundSpec *spec);
extern Oid get_partition_parent(Oid relid); extern Oid get_partition_parent(Oid relid);
extern List *get_partition_ancestors(Oid relid);
extern List *get_qual_from_partbound(Relation rel, Relation parent, extern List *get_qual_from_partbound(Relation rel, Relation parent,
PartitionBoundSpec *spec); PartitionBoundSpec *spec);
extern List *map_partition_varattnos(List *expr, int fromrel_varno, extern List *map_partition_varattnos(List *expr, int fromrel_varno,

View File

@@ -362,6 +362,20 @@ typedef struct JunkFilter
AttrNumber jf_junkAttNo; AttrNumber jf_junkAttNo;
} JunkFilter; } JunkFilter;
/*
* OnConflictSetState
*
* Executor state of an ON CONFLICT DO UPDATE operation.
*/
typedef struct OnConflictSetState
{
NodeTag type;
ProjectionInfo *oc_ProjInfo; /* for ON CONFLICT DO UPDATE SET */
TupleDesc oc_ProjTupdesc; /* TupleDesc for the above projection */
ExprState *oc_WhereClause; /* state for the WHERE clause */
} OnConflictSetState;
/* /*
* ResultRelInfo * ResultRelInfo
* *
@@ -424,11 +438,11 @@ typedef struct ResultRelInfo
/* for computing a RETURNING list */ /* for computing a RETURNING list */
ProjectionInfo *ri_projectReturning; ProjectionInfo *ri_projectReturning;
/* for computing ON CONFLICT DO UPDATE SET */ /* list of arbiter indexes to use to check conflicts */
ProjectionInfo *ri_onConflictSetProj; List *ri_onConflictArbiterIndexes;
/* list of ON CONFLICT DO UPDATE exprs (qual) */ /* ON CONFLICT evaluation state */
ExprState *ri_onConflictSetWhere; OnConflictSetState *ri_onConflict;
/* partition check expression */ /* partition check expression */
List *ri_PartitionCheck; List *ri_PartitionCheck;

View File

@@ -34,6 +34,7 @@ typedef enum NodeTag
T_ExprContext, T_ExprContext,
T_ProjectionInfo, T_ProjectionInfo,
T_JunkFilter, T_JunkFilter,
T_OnConflictSetState,
T_ResultRelInfo, T_ResultRelInfo,
T_EState, T_EState,
T_TupleTableSlot, T_TupleTableSlot,

View File

@@ -786,16 +786,102 @@ select * from selfconflict;
(3 rows) (3 rows)
drop table selfconflict; drop table selfconflict;
-- check that the following works: -- check ON CONFLICT handling with partitioned tables
-- insert into partitioned_table on conflict do nothing create table parted_conflict_test (a int unique, b char) partition by list (a);
create table parted_conflict_test (a int, b char) partition by list (a); create table parted_conflict_test_1 partition of parted_conflict_test (b unique) for values in (1, 2);
create table parted_conflict_test_1 partition of parted_conflict_test (b unique) for values in (1); -- no indexes required here
insert into parted_conflict_test values (1, 'a') on conflict do nothing; insert into parted_conflict_test values (1, 'a') on conflict do nothing;
insert into parted_conflict_test values (1, 'a') on conflict do nothing; -- index on a required, which does exist in parent
-- however, on conflict do update is not supported yet insert into parted_conflict_test values (1, 'a') on conflict (a) do nothing;
insert into parted_conflict_test values (1) on conflict (b) do update set a = excluded.a; insert into parted_conflict_test values (1, 'a') on conflict (a) do update set b = excluded.b;
ERROR: ON CONFLICT DO UPDATE cannot be applied to partitioned table "parted_conflict_test" -- targeting partition directly will work
-- but it works OK if we target the partition directly insert into parted_conflict_test_1 values (1, 'a') on conflict (a) do nothing;
insert into parted_conflict_test_1 values (1) on conflict (b) do insert into parted_conflict_test_1 values (1, 'b') on conflict (a) do update set b = excluded.b;
update set a = excluded.a; -- index on b required, which doesn't exist in parent
insert into parted_conflict_test values (2, 'b') on conflict (b) do update set a = excluded.a;
ERROR: there is no unique or exclusion constraint matching the ON CONFLICT specification
-- targeting partition directly will work
insert into parted_conflict_test_1 values (2, 'b') on conflict (b) do update set a = excluded.a;
-- should see (2, 'b')
select * from parted_conflict_test order by a;
a | b
---+---
2 | b
(1 row)
-- now check that DO UPDATE works correctly for target partition with
-- different attribute numbers
create table parted_conflict_test_2 (b char, a int unique);
alter table parted_conflict_test attach partition parted_conflict_test_2 for values in (3);
truncate parted_conflict_test;
insert into parted_conflict_test values (3, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test values (3, 'b') on conflict (a) do update set b = excluded.b;
-- should see (3, 'b')
select * from parted_conflict_test order by a;
a | b
---+---
3 | b
(1 row)
-- case where parent will have a dropped column, but the partition won't
alter table parted_conflict_test drop b, add b char;
create table parted_conflict_test_3 partition of parted_conflict_test for values in (4);
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (4, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test (a, b) values (4, 'b') on conflict (a) do update set b = excluded.b where parted_conflict_test.b = 'a';
-- should see (4, 'b')
select * from parted_conflict_test order by a;
a | b
---+---
4 | b
(1 row)
-- case with multi-level partitioning
create table parted_conflict_test_4 partition of parted_conflict_test for values in (5) partition by list (a);
create table parted_conflict_test_4_1 partition of parted_conflict_test_4 for values in (5);
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (5, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test (a, b) values (5, 'b') on conflict (a) do update set b = excluded.b where parted_conflict_test.b = 'a';
-- should see (5, 'b')
select * from parted_conflict_test order by a;
a | b
---+---
5 | b
(1 row)
-- test with multiple rows
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (1, 'a'), (2, 'a'), (4, 'a') on conflict (a) do update set b = excluded.b where excluded.b = 'b';
insert into parted_conflict_test (a, b) values (1, 'b'), (2, 'c'), (4, 'b') on conflict (a) do update set b = excluded.b where excluded.b = 'b';
-- should see (1, 'b'), (2, 'a'), (4, 'b')
select * from parted_conflict_test order by a;
a | b
---+---
1 | b
2 | a
4 | b
(3 rows)
drop table parted_conflict_test; drop table parted_conflict_test;
-- test behavior of inserting a conflicting tuple into an intermediate
-- partitioning level
create table parted_conflict (a int primary key, b text) partition by range (a);
create table parted_conflict_1 partition of parted_conflict for values from (0) to (1000) partition by range (a);
create table parted_conflict_1_1 partition of parted_conflict_1 for values from (0) to (500);
insert into parted_conflict values (40, 'forty');
insert into parted_conflict_1 values (40, 'cuarenta')
on conflict (a) do update set b = excluded.b;
drop table parted_conflict;
-- same thing, but this time try to use an index that's created not in the
-- partition
create table parted_conflict (a int, b text) partition by range (a);
create table parted_conflict_1 partition of parted_conflict for values from (0) to (1000) partition by range (a);
create unique index on only parted_conflict_1 (a);
create unique index on only parted_conflict (a);
alter index parted_conflict_a_idx attach partition parted_conflict_1_a_idx;
create table parted_conflict_1_1 partition of parted_conflict_1 for values from (0) to (500);
insert into parted_conflict values (40, 'forty');
insert into parted_conflict_1 values (40, 'cuarenta')
on conflict (a) do update set b = excluded.b;
ERROR: there is no unique or exclusion constraint matching the ON CONFLICT specification
drop table parted_conflict;

View File

@@ -2624,6 +2624,39 @@ insert into my_table values (3, 'CCC'), (4, 'DDD')
NOTICE: trigger = my_table_update_trig, old table = (3,CCC), (4,DDD), new table = (3,CCC:CCC), (4,DDD:DDD) NOTICE: trigger = my_table_update_trig, old table = (3,CCC), (4,DDD), new table = (3,CCC:CCC), (4,DDD:DDD)
NOTICE: trigger = my_table_insert_trig, new table = <NULL> NOTICE: trigger = my_table_insert_trig, new table = <NULL>
-- --
-- now using a partitioned table
--
create table iocdu_tt_parted (a int primary key, b text) partition by list (a);
create table iocdu_tt_parted1 partition of iocdu_tt_parted for values in (1);
create table iocdu_tt_parted2 partition of iocdu_tt_parted for values in (2);
create table iocdu_tt_parted3 partition of iocdu_tt_parted for values in (3);
create table iocdu_tt_parted4 partition of iocdu_tt_parted for values in (4);
create trigger iocdu_tt_parted_insert_trig
after insert on iocdu_tt_parted referencing new table as new_table
for each statement execute procedure dump_insert();
create trigger iocdu_tt_parted_update_trig
after update on iocdu_tt_parted referencing old table as old_table new table as new_table
for each statement execute procedure dump_update();
-- inserts only
insert into iocdu_tt_parted values (1, 'AAA'), (2, 'BBB')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
NOTICE: trigger = iocdu_tt_parted_update_trig, old table = <NULL>, new table = <NULL>
NOTICE: trigger = iocdu_tt_parted_insert_trig, new table = (1,AAA), (2,BBB)
-- mixture of inserts and updates
insert into iocdu_tt_parted values (1, 'AAA'), (2, 'BBB'), (3, 'CCC'), (4, 'DDD')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
NOTICE: trigger = iocdu_tt_parted_update_trig, old table = (1,AAA), (2,BBB), new table = (1,AAA:AAA), (2,BBB:BBB)
NOTICE: trigger = iocdu_tt_parted_insert_trig, new table = (3,CCC), (4,DDD)
-- updates only
insert into iocdu_tt_parted values (3, 'CCC'), (4, 'DDD')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
NOTICE: trigger = iocdu_tt_parted_update_trig, old table = (3,CCC), (4,DDD), new table = (3,CCC:CCC), (4,DDD:DDD)
NOTICE: trigger = iocdu_tt_parted_insert_trig, new table = <NULL>
drop table iocdu_tt_parted;
--
-- Verify that you can't create a trigger with transition tables for -- Verify that you can't create a trigger with transition tables for
-- more than one event. -- more than one event.
-- --

View File

@@ -472,15 +472,90 @@ select * from selfconflict;
drop table selfconflict; drop table selfconflict;
-- check that the following works: -- check ON CONFLICT handling with partitioned tables
-- insert into partitioned_table on conflict do nothing create table parted_conflict_test (a int unique, b char) partition by list (a);
create table parted_conflict_test (a int, b char) partition by list (a); create table parted_conflict_test_1 partition of parted_conflict_test (b unique) for values in (1, 2);
create table parted_conflict_test_1 partition of parted_conflict_test (b unique) for values in (1);
-- no indexes required here
insert into parted_conflict_test values (1, 'a') on conflict do nothing; insert into parted_conflict_test values (1, 'a') on conflict do nothing;
insert into parted_conflict_test values (1, 'a') on conflict do nothing;
-- however, on conflict do update is not supported yet -- index on a required, which does exist in parent
insert into parted_conflict_test values (1) on conflict (b) do update set a = excluded.a; insert into parted_conflict_test values (1, 'a') on conflict (a) do nothing;
-- but it works OK if we target the partition directly insert into parted_conflict_test values (1, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test_1 values (1) on conflict (b) do
update set a = excluded.a; -- targeting partition directly will work
insert into parted_conflict_test_1 values (1, 'a') on conflict (a) do nothing;
insert into parted_conflict_test_1 values (1, 'b') on conflict (a) do update set b = excluded.b;
-- index on b required, which doesn't exist in parent
insert into parted_conflict_test values (2, 'b') on conflict (b) do update set a = excluded.a;
-- targeting partition directly will work
insert into parted_conflict_test_1 values (2, 'b') on conflict (b) do update set a = excluded.a;
-- should see (2, 'b')
select * from parted_conflict_test order by a;
-- now check that DO UPDATE works correctly for target partition with
-- different attribute numbers
create table parted_conflict_test_2 (b char, a int unique);
alter table parted_conflict_test attach partition parted_conflict_test_2 for values in (3);
truncate parted_conflict_test;
insert into parted_conflict_test values (3, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test values (3, 'b') on conflict (a) do update set b = excluded.b;
-- should see (3, 'b')
select * from parted_conflict_test order by a;
-- case where parent will have a dropped column, but the partition won't
alter table parted_conflict_test drop b, add b char;
create table parted_conflict_test_3 partition of parted_conflict_test for values in (4);
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (4, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test (a, b) values (4, 'b') on conflict (a) do update set b = excluded.b where parted_conflict_test.b = 'a';
-- should see (4, 'b')
select * from parted_conflict_test order by a;
-- case with multi-level partitioning
create table parted_conflict_test_4 partition of parted_conflict_test for values in (5) partition by list (a);
create table parted_conflict_test_4_1 partition of parted_conflict_test_4 for values in (5);
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (5, 'a') on conflict (a) do update set b = excluded.b;
insert into parted_conflict_test (a, b) values (5, 'b') on conflict (a) do update set b = excluded.b where parted_conflict_test.b = 'a';
-- should see (5, 'b')
select * from parted_conflict_test order by a;
-- test with multiple rows
truncate parted_conflict_test;
insert into parted_conflict_test (a, b) values (1, 'a'), (2, 'a'), (4, 'a') on conflict (a) do update set b = excluded.b where excluded.b = 'b';
insert into parted_conflict_test (a, b) values (1, 'b'), (2, 'c'), (4, 'b') on conflict (a) do update set b = excluded.b where excluded.b = 'b';
-- should see (1, 'b'), (2, 'a'), (4, 'b')
select * from parted_conflict_test order by a;
drop table parted_conflict_test; drop table parted_conflict_test;
-- test behavior of inserting a conflicting tuple into an intermediate
-- partitioning level
create table parted_conflict (a int primary key, b text) partition by range (a);
create table parted_conflict_1 partition of parted_conflict for values from (0) to (1000) partition by range (a);
create table parted_conflict_1_1 partition of parted_conflict_1 for values from (0) to (500);
insert into parted_conflict values (40, 'forty');
insert into parted_conflict_1 values (40, 'cuarenta')
on conflict (a) do update set b = excluded.b;
drop table parted_conflict;
-- same thing, but this time try to use an index that's created not in the
-- partition
create table parted_conflict (a int, b text) partition by range (a);
create table parted_conflict_1 partition of parted_conflict for values from (0) to (1000) partition by range (a);
create unique index on only parted_conflict_1 (a);
create unique index on only parted_conflict (a);
alter index parted_conflict_a_idx attach partition parted_conflict_1_a_idx;
create table parted_conflict_1_1 partition of parted_conflict_1 for values from (0) to (500);
insert into parted_conflict values (40, 'forty');
insert into parted_conflict_1 values (40, 'cuarenta')
on conflict (a) do update set b = excluded.b;
drop table parted_conflict;

View File

@@ -1982,6 +1982,39 @@ insert into my_table values (3, 'CCC'), (4, 'DDD')
on conflict (a) do on conflict (a) do
update set b = my_table.b || ':' || excluded.b; update set b = my_table.b || ':' || excluded.b;
--
-- now using a partitioned table
--
create table iocdu_tt_parted (a int primary key, b text) partition by list (a);
create table iocdu_tt_parted1 partition of iocdu_tt_parted for values in (1);
create table iocdu_tt_parted2 partition of iocdu_tt_parted for values in (2);
create table iocdu_tt_parted3 partition of iocdu_tt_parted for values in (3);
create table iocdu_tt_parted4 partition of iocdu_tt_parted for values in (4);
create trigger iocdu_tt_parted_insert_trig
after insert on iocdu_tt_parted referencing new table as new_table
for each statement execute procedure dump_insert();
create trigger iocdu_tt_parted_update_trig
after update on iocdu_tt_parted referencing old table as old_table new table as new_table
for each statement execute procedure dump_update();
-- inserts only
insert into iocdu_tt_parted values (1, 'AAA'), (2, 'BBB')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
-- mixture of inserts and updates
insert into iocdu_tt_parted values (1, 'AAA'), (2, 'BBB'), (3, 'CCC'), (4, 'DDD')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
-- updates only
insert into iocdu_tt_parted values (3, 'CCC'), (4, 'DDD')
on conflict (a) do
update set b = iocdu_tt_parted.b || ':' || excluded.b;
drop table iocdu_tt_parted;
-- --
-- Verify that you can't create a trigger with transition tables for -- Verify that you can't create a trigger with transition tables for
-- more than one event. -- more than one event.